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 #ifdef GFLAGS
11 #ifdef NUMA
12 #include <numa.h>
13 #include <numaif.h>
14 #endif
15 #ifndef OS_WIN
16 #include <unistd.h>
17 #endif
18 #include <fcntl.h>
19 #include <stdio.h>
20 #include <stdlib.h>
21 #include <sys/types.h>
22 #include <atomic>
23 #include <cinttypes>
24 #include <condition_variable>
25 #include <cstddef>
26 #include <memory>
27 #include <mutex>
28 #include <thread>
29 #include <unordered_map>
30 
31 #include "db/db_impl/db_impl.h"
32 #include "db/malloc_stats.h"
33 #include "db/version_set.h"
34 #include "hdfs/env_hdfs.h"
35 #include "monitoring/histogram.h"
36 #include "monitoring/statistics.h"
37 #include "options/cf_options.h"
38 #include "port/port.h"
39 #include "port/stack_trace.h"
40 #include "rocksdb/cache.h"
41 #include "rocksdb/db.h"
42 #include "rocksdb/env.h"
43 #include "rocksdb/filter_policy.h"
44 #include "rocksdb/memtablerep.h"
45 #include "rocksdb/options.h"
46 #include "rocksdb/perf_context.h"
47 #include "rocksdb/persistent_cache.h"
48 #include "rocksdb/rate_limiter.h"
49 #include "rocksdb/slice.h"
50 #include "rocksdb/slice_transform.h"
51 #include "rocksdb/stats_history.h"
52 #include "rocksdb/utilities/object_registry.h"
53 #include "rocksdb/utilities/optimistic_transaction_db.h"
54 #include "rocksdb/utilities/options_util.h"
55 #include "rocksdb/utilities/sim_cache.h"
56 #include "rocksdb/utilities/transaction.h"
57 #include "rocksdb/utilities/transaction_db.h"
58 #include "rocksdb/write_batch.h"
59 #include "test_util/testutil.h"
60 #include "test_util/transaction_test_util.h"
61 #include "util/cast_util.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/stderr_logger.h"
68 #include "util/string_util.h"
69 #include "util/xxhash.h"
70 #include "utilities/blob_db/blob_db.h"
71 #include "utilities/merge_operators.h"
72 #include "utilities/merge_operators/bytesxor.h"
73 #include "utilities/merge_operators/sortlist.h"
74 #include "utilities/persistent_cache/block_cache_tier.h"
75 
76 #ifdef OS_WIN
77 #include <io.h>  // open/close
78 #endif
79 
80 using GFLAGS_NAMESPACE::ParseCommandLineFlags;
81 using GFLAGS_NAMESPACE::RegisterFlagValidator;
82 using GFLAGS_NAMESPACE::SetUsageMessage;
83 
84 DEFINE_string(
85     benchmarks,
86     "fillseq,"
87     "fillseqdeterministic,"
88     "fillsync,"
89     "fillrandom,"
90     "filluniquerandomdeterministic,"
91     "overwrite,"
92     "readrandom,"
93     "newiterator,"
94     "newiteratorwhilewriting,"
95     "seekrandom,"
96     "seekrandomwhilewriting,"
97     "seekrandomwhilemerging,"
98     "readseq,"
99     "readreverse,"
100     "compact,"
101     "compactall,"
102     "multireadrandom,"
103     "mixgraph,"
104     "readseq,"
105     "readtorowcache,"
106     "readtocache,"
107     "readreverse,"
108     "readwhilewriting,"
109     "readwhilemerging,"
110     "readwhilescanning,"
111     "readrandomwriterandom,"
112     "updaterandom,"
113     "xorupdaterandom,"
114     "randomwithverify,"
115     "fill100K,"
116     "crc32c,"
117     "xxhash,"
118     "compress,"
119     "uncompress,"
120     "acquireload,"
121     "fillseekseq,"
122     "randomtransaction,"
123     "randomreplacekeys,"
124     "timeseries,"
125     "getmergeoperands",
126 
127     "Comma-separated list of operations to run in the specified"
128     " order. Available benchmarks:\n"
129     "\tfillseq       -- write N values in sequential key"
130     " order in async mode\n"
131     "\tfillseqdeterministic       -- write N values in the specified"
132     " key order and keep the shape of the LSM tree\n"
133     "\tfillrandom    -- write N values in random key order in async"
134     " mode\n"
135     "\tfilluniquerandomdeterministic       -- write N values in a random"
136     " key order and keep the shape of the LSM tree\n"
137     "\toverwrite     -- overwrite N values in random key order in"
138     " async mode\n"
139     "\tfillsync      -- write N/1000 values in random key order in "
140     "sync mode\n"
141     "\tfill100K      -- write N/1000 100K values in random order in"
142     " async mode\n"
143     "\tdeleteseq     -- delete N keys in sequential order\n"
144     "\tdeleterandom  -- delete N keys in random order\n"
145     "\treadseq       -- read N times sequentially\n"
146     "\treadtocache   -- 1 thread reading database sequentially\n"
147     "\treadreverse   -- read N times in reverse order\n"
148     "\treadrandom    -- read N times in random order\n"
149     "\treadmissing   -- read N missing keys in random order\n"
150     "\treadwhilewriting      -- 1 writer, N threads doing random "
151     "reads\n"
152     "\treadwhilemerging      -- 1 merger, N threads doing random "
153     "reads\n"
154     "\treadwhilescanning     -- 1 thread doing full table scan, "
155     "N threads doing random reads\n"
156     "\treadrandomwriterandom -- N threads doing random-read, "
157     "random-write\n"
158     "\tupdaterandom  -- N threads doing read-modify-write for random "
159     "keys\n"
160     "\txorupdaterandom  -- N threads doing read-XOR-write for "
161     "random keys\n"
162     "\tappendrandom  -- N threads doing read-modify-write with "
163     "growing values\n"
164     "\tmergerandom   -- same as updaterandom/appendrandom using merge"
165     " operator. "
166     "Must be used with merge_operator\n"
167     "\treadrandommergerandom -- perform N random read-or-merge "
168     "operations. Must be used with merge_operator\n"
169     "\tnewiterator   -- repeated iterator creation\n"
170     "\tseekrandom    -- N random seeks, call Next seek_nexts times "
171     "per seek\n"
172     "\tseekrandomwhilewriting -- seekrandom and 1 thread doing "
173     "overwrite\n"
174     "\tseekrandomwhilemerging -- seekrandom and 1 thread doing "
175     "merge\n"
176     "\tcrc32c        -- repeated crc32c of 4K of data\n"
177     "\txxhash        -- repeated xxHash of 4K of data\n"
178     "\tacquireload   -- load N*1000 times\n"
179     "\tfillseekseq   -- write N values in sequential key, then read "
180     "them by seeking to each key\n"
181     "\trandomtransaction     -- execute N random transactions and "
182     "verify correctness\n"
183     "\trandomreplacekeys     -- randomly replaces N keys by deleting "
184     "the old version and putting the new version\n\n"
185     "\ttimeseries            -- 1 writer generates time series data "
186     "and multiple readers doing random reads on id\n\n"
187     "Meta operations:\n"
188     "\tcompact     -- Compact the entire DB; If multiple, randomly choose one\n"
189     "\tcompactall  -- Compact the entire DB\n"
190     "\tstats       -- Print DB stats\n"
191     "\tresetstats  -- Reset DB stats\n"
192     "\tlevelstats  -- Print the number of files and bytes per level\n"
193     "\tsstables    -- Print sstable info\n"
194     "\theapprofile -- Dump a heap profile (if supported by this port)\n"
195     "\treplay      -- replay the trace file specified with trace_file\n"
196     "\tgetmergeoperands -- Insert lots of merge records which are a list of "
197     "sorted ints for a key and then compare performance of lookup for another "
198     "key "
199     "by doing a Get followed by binary searching in the large sorted list vs "
200     "doing a GetMergeOperands and binary searching in the operands which are"
201     "sorted sub-lists. The MergeOperator used is sortlist.h\n");
202 
203 DEFINE_int64(num, 1000000, "Number of key/values to place in database");
204 
205 DEFINE_int64(numdistinct, 1000,
206              "Number of distinct keys to use. Used in RandomWithVerify to "
207              "read/write on fewer keys so that gets are more likely to find the"
208              " key and puts are more likely to update the same key");
209 
210 DEFINE_int64(merge_keys, -1,
211              "Number of distinct keys to use for MergeRandom and "
212              "ReadRandomMergeRandom. "
213              "If negative, there will be FLAGS_num keys.");
214 DEFINE_int32(num_column_families, 1, "Number of Column Families to use.");
215 
216 DEFINE_int32(
217     num_hot_column_families, 0,
218     "Number of Hot Column Families. If more than 0, only write to this "
219     "number of column families. After finishing all the writes to them, "
220     "create new set of column families and insert to them. Only used "
221     "when num_column_families > 1.");
222 
223 DEFINE_string(column_family_distribution, "",
224               "Comma-separated list of percentages, where the ith element "
225               "indicates the probability of an op using the ith column family. "
226               "The number of elements must be `num_hot_column_families` if "
227               "specified; otherwise, it must be `num_column_families`. The "
228               "sum of elements must be 100. E.g., if `num_column_families=4`, "
229               "and `num_hot_column_families=0`, a valid list could be "
230               "\"10,20,30,40\".");
231 
232 DEFINE_int64(reads, -1, "Number of read operations to do.  "
233              "If negative, do FLAGS_num reads.");
234 
235 DEFINE_int64(deletes, -1, "Number of delete operations to do.  "
236              "If negative, do FLAGS_num deletions.");
237 
238 DEFINE_int32(bloom_locality, 0, "Control bloom filter probes locality");
239 
240 DEFINE_int64(seed, 0, "Seed base for random number generators. "
241              "When 0 it is deterministic.");
242 
243 DEFINE_int32(threads, 1, "Number of concurrent threads to run.");
244 
245 DEFINE_int32(duration, 0, "Time in seconds for the random-ops tests to run."
246              " When 0 then num & reads determine the test duration");
247 
248 DEFINE_string(value_size_distribution_type, "fixed",
249               "Value size distribution type: fixed, uniform, normal");
250 
251 DEFINE_int32(value_size, 100, "Size of each value in fixed distribution");
252 static unsigned int value_size = 100;
253 
254 DEFINE_int32(value_size_min, 100, "Min size of random value");
255 
256 DEFINE_int32(value_size_max, 102400, "Max size of random value");
257 
258 DEFINE_int32(seek_nexts, 0,
259              "How many times to call Next() after Seek() in "
260              "fillseekseq, seekrandom, seekrandomwhilewriting and "
261              "seekrandomwhilemerging");
262 
263 DEFINE_bool(reverse_iterator, false,
264             "When true use Prev rather than Next for iterators that do "
265             "Seek and then Next");
266 
267 DEFINE_int64(max_scan_distance, 0,
268              "Used to define iterate_upper_bound (or iterate_lower_bound "
269              "if FLAGS_reverse_iterator is set to true) when value is nonzero");
270 
271 DEFINE_bool(use_uint64_comparator, false, "use Uint64 user comparator");
272 
273 DEFINE_int64(batch_size, 1, "Batch size");
274 
ValidateKeySize(const char *,int32_t)275 static bool ValidateKeySize(const char* /*flagname*/, int32_t /*value*/) {
276   return true;
277 }
278 
ValidateUint32Range(const char * flagname,uint64_t value)279 static bool ValidateUint32Range(const char* flagname, uint64_t value) {
280   if (value > std::numeric_limits<uint32_t>::max()) {
281     fprintf(stderr, "Invalid value for --%s: %lu, overflow\n", flagname,
282             (unsigned long)value);
283     return false;
284   }
285   return true;
286 }
287 
288 DEFINE_int32(key_size, 16, "size of each key");
289 
290 DEFINE_int32(num_multi_db, 0,
291              "Number of DBs used in the benchmark. 0 means single DB.");
292 
293 DEFINE_double(compression_ratio, 0.5, "Arrange to generate values that shrink"
294               " to this fraction of their original size after compression");
295 
296 DEFINE_double(read_random_exp_range, 0.0,
297               "Read random's key will be generated using distribution of "
298               "num * exp(-r) where r is uniform number from 0 to this value. "
299               "The larger the number is, the more skewed the reads are. "
300               "Only used in readrandom and multireadrandom benchmarks.");
301 
302 DEFINE_bool(histogram, false, "Print histogram of operation timings");
303 
304 DEFINE_bool(enable_numa, false,
305             "Make operations aware of NUMA architecture and bind memory "
306             "and cpus corresponding to nodes together. In NUMA, memory "
307             "in same node as CPUs are closer when compared to memory in "
308             "other nodes. Reads can be faster when the process is bound to "
309             "CPU and memory of same node. Use \"$numactl --hardware\" command "
310             "to see NUMA memory architecture.");
311 
312 DEFINE_int64(db_write_buffer_size,
313              ROCKSDB_NAMESPACE::Options().db_write_buffer_size,
314              "Number of bytes to buffer in all memtables before compacting");
315 
316 DEFINE_bool(cost_write_buffer_to_cache, false,
317             "The usage of memtable is costed to the block cache");
318 
319 DEFINE_int64(write_buffer_size, ROCKSDB_NAMESPACE::Options().write_buffer_size,
320              "Number of bytes to buffer in memtable before compacting");
321 
322 DEFINE_int32(max_write_buffer_number,
323              ROCKSDB_NAMESPACE::Options().max_write_buffer_number,
324              "The number of in-memory memtables. Each memtable is of size"
325              " write_buffer_size bytes.");
326 
327 DEFINE_int32(min_write_buffer_number_to_merge,
328              ROCKSDB_NAMESPACE::Options().min_write_buffer_number_to_merge,
329              "The minimum number of write buffers that will be merged together"
330              "before writing to storage. This is cheap because it is an"
331              "in-memory merge. If this feature is not enabled, then all these"
332              "write buffers are flushed to L0 as separate files and this "
333              "increases read amplification because a get request has to check"
334              " in all of these files. Also, an in-memory merge may result in"
335              " writing less data to storage if there are duplicate records "
336              " in each of these individual write buffers.");
337 
338 DEFINE_int32(max_write_buffer_number_to_maintain,
339              ROCKSDB_NAMESPACE::Options().max_write_buffer_number_to_maintain,
340              "The total maximum number of write buffers to maintain in memory "
341              "including copies of buffers that have already been flushed. "
342              "Unlike max_write_buffer_number, this parameter does not affect "
343              "flushing. This controls the minimum amount of write history "
344              "that will be available in memory for conflict checking when "
345              "Transactions are used. If this value is too low, some "
346              "transactions may fail at commit time due to not being able to "
347              "determine whether there were any write conflicts. Setting this "
348              "value to 0 will cause write buffers to be freed immediately "
349              "after they are flushed.  If this value is set to -1, "
350              "'max_write_buffer_number' will be used.");
351 
352 DEFINE_int64(max_write_buffer_size_to_maintain,
353              ROCKSDB_NAMESPACE::Options().max_write_buffer_size_to_maintain,
354              "The total maximum size of write buffers to maintain in memory "
355              "including copies of buffers that have already been flushed. "
356              "Unlike max_write_buffer_number, this parameter does not affect "
357              "flushing. This controls the minimum amount of write history "
358              "that will be available in memory for conflict checking when "
359              "Transactions are used. If this value is too low, some "
360              "transactions may fail at commit time due to not being able to "
361              "determine whether there were any write conflicts. Setting this "
362              "value to 0 will cause write buffers to be freed immediately "
363              "after they are flushed.  If this value is set to -1, "
364              "'max_write_buffer_number' will be used.");
365 
366 DEFINE_int32(max_background_jobs,
367              ROCKSDB_NAMESPACE::Options().max_background_jobs,
368              "The maximum number of concurrent background jobs that can occur "
369              "in parallel.");
370 
371 DEFINE_int32(num_bottom_pri_threads, 0,
372              "The number of threads in the bottom-priority thread pool (used "
373              "by universal compaction only).");
374 
375 DEFINE_int32(num_high_pri_threads, 0,
376              "The maximum number of concurrent background compactions"
377              " that can occur in parallel.");
378 
379 DEFINE_int32(num_low_pri_threads, 0,
380              "The maximum number of concurrent background compactions"
381              " that can occur in parallel.");
382 
383 DEFINE_int32(max_background_compactions,
384              ROCKSDB_NAMESPACE::Options().max_background_compactions,
385              "The maximum number of concurrent background compactions"
386              " that can occur in parallel.");
387 
388 DEFINE_int32(base_background_compactions, -1, "DEPRECATED");
389 
390 DEFINE_uint64(subcompactions, 1,
391               "Maximum number of subcompactions to divide L0-L1 compactions "
392               "into.");
393 static const bool FLAGS_subcompactions_dummy
394     __attribute__((__unused__)) = RegisterFlagValidator(&FLAGS_subcompactions,
395                                                     &ValidateUint32Range);
396 
397 DEFINE_int32(max_background_flushes,
398              ROCKSDB_NAMESPACE::Options().max_background_flushes,
399              "The maximum number of concurrent background flushes"
400              " that can occur in parallel.");
401 
402 static ROCKSDB_NAMESPACE::CompactionStyle FLAGS_compaction_style_e;
403 DEFINE_int32(compaction_style,
404              (int32_t)ROCKSDB_NAMESPACE::Options().compaction_style,
405              "style of compaction: level-based, universal and fifo");
406 
407 static ROCKSDB_NAMESPACE::CompactionPri FLAGS_compaction_pri_e;
408 DEFINE_int32(compaction_pri,
409              (int32_t)ROCKSDB_NAMESPACE::Options().compaction_pri,
410              "priority of files to compaction: by size or by data age");
411 
412 DEFINE_int32(universal_size_ratio, 0,
413              "Percentage flexibility while comparing file size"
414              " (for universal compaction only).");
415 
416 DEFINE_int32(universal_min_merge_width, 0, "The minimum number of files in a"
417              " single compaction run (for universal compaction only).");
418 
419 DEFINE_int32(universal_max_merge_width, 0, "The max number of files to compact"
420              " in universal style compaction");
421 
422 DEFINE_int32(universal_max_size_amplification_percent, 0,
423              "The max size amplification for universal style compaction");
424 
425 DEFINE_int32(universal_compression_size_percent, -1,
426              "The percentage of the database to compress for universal "
427              "compaction. -1 means compress everything.");
428 
429 DEFINE_bool(universal_allow_trivial_move, false,
430             "Allow trivial move in universal compaction.");
431 
432 DEFINE_int64(cache_size, 8 << 20,  // 8MB
433              "Number of bytes to use as a cache of uncompressed data");
434 
435 DEFINE_int32(cache_numshardbits, 6,
436              "Number of shards for the block cache"
437              " is 2 ** cache_numshardbits. Negative means use default settings."
438              " This is applied only if FLAGS_cache_size is non-negative.");
439 
440 DEFINE_double(cache_high_pri_pool_ratio, 0.0,
441               "Ratio of block cache reserve for high pri blocks. "
442               "If > 0.0, we also enable "
443               "cache_index_and_filter_blocks_with_high_priority.");
444 
445 DEFINE_bool(use_clock_cache, false,
446             "Replace default LRU block cache with clock cache.");
447 
448 DEFINE_int64(simcache_size, -1,
449              "Number of bytes to use as a simcache of "
450              "uncompressed data. Nagative value disables simcache.");
451 
452 DEFINE_bool(cache_index_and_filter_blocks, false,
453             "Cache index/filter blocks in block cache.");
454 
455 DEFINE_bool(partition_index_and_filters, false,
456             "Partition index and filter blocks.");
457 
458 DEFINE_bool(partition_index, false, "Partition index blocks");
459 
460 DEFINE_bool(index_with_first_key, false, "Include first key in the index");
461 
462 DEFINE_int64(
463     index_shortening_mode, 2,
464     "mode to shorten index: 0 for no shortening; 1 for only shortening "
465     "separaters; 2 for shortening shortening and successor");
466 
467 DEFINE_int64(metadata_block_size,
468              ROCKSDB_NAMESPACE::BlockBasedTableOptions().metadata_block_size,
469              "Max partition size when partitioning index/filters");
470 
471 // The default reduces the overhead of reading time with flash. With HDD, which
472 // offers much less throughput, however, this number better to be set to 1.
473 DEFINE_int32(ops_between_duration_checks, 1000,
474              "Check duration limit every x ops");
475 
476 DEFINE_bool(pin_l0_filter_and_index_blocks_in_cache, false,
477             "Pin index/filter blocks of L0 files in block cache.");
478 
479 DEFINE_bool(
480     pin_top_level_index_and_filter, false,
481     "Pin top-level index of partitioned index/filter blocks in block cache.");
482 
483 DEFINE_int32(block_size,
484              static_cast<int32_t>(
485                  ROCKSDB_NAMESPACE::BlockBasedTableOptions().block_size),
486              "Number of bytes in a block.");
487 
488 DEFINE_int32(format_version,
489              static_cast<int32_t>(
490                  ROCKSDB_NAMESPACE::BlockBasedTableOptions().format_version),
491              "Format version of SST files.");
492 
493 DEFINE_int32(block_restart_interval,
494              ROCKSDB_NAMESPACE::BlockBasedTableOptions().block_restart_interval,
495              "Number of keys between restart points "
496              "for delta encoding of keys in data block.");
497 
498 DEFINE_int32(
499     index_block_restart_interval,
500     ROCKSDB_NAMESPACE::BlockBasedTableOptions().index_block_restart_interval,
501     "Number of keys between restart points "
502     "for delta encoding of keys in index block.");
503 
504 DEFINE_int32(read_amp_bytes_per_bit,
505              ROCKSDB_NAMESPACE::BlockBasedTableOptions().read_amp_bytes_per_bit,
506              "Number of bytes per bit to be used in block read-amp bitmap");
507 
508 DEFINE_bool(
509     enable_index_compression,
510     ROCKSDB_NAMESPACE::BlockBasedTableOptions().enable_index_compression,
511     "Compress the index block");
512 
513 DEFINE_bool(block_align,
514             ROCKSDB_NAMESPACE::BlockBasedTableOptions().block_align,
515             "Align data blocks on page size");
516 
517 DEFINE_bool(use_data_block_hash_index, false,
518             "if use kDataBlockBinaryAndHash "
519             "instead of kDataBlockBinarySearch. "
520             "This is valid if only we use BlockTable");
521 
522 DEFINE_double(data_block_hash_table_util_ratio, 0.75,
523               "util ratio for data block hash index table. "
524               "This is only valid if use_data_block_hash_index is "
525               "set to true");
526 
527 DEFINE_int64(compressed_cache_size, -1,
528              "Number of bytes to use as a cache of compressed data.");
529 
530 DEFINE_int64(row_cache_size, 0,
531              "Number of bytes to use as a cache of individual rows"
532              " (0 = disabled).");
533 
534 DEFINE_int32(open_files, ROCKSDB_NAMESPACE::Options().max_open_files,
535              "Maximum number of files to keep open at the same time"
536              " (use default if == 0)");
537 
538 DEFINE_int32(file_opening_threads,
539              ROCKSDB_NAMESPACE::Options().max_file_opening_threads,
540              "If open_files is set to -1, this option set the number of "
541              "threads that will be used to open files during DB::Open()");
542 
543 DEFINE_bool(new_table_reader_for_compaction_inputs, true,
544              "If true, uses a separate file handle for compaction inputs");
545 
546 DEFINE_int32(compaction_readahead_size, 0, "Compaction readahead size");
547 
548 DEFINE_int32(log_readahead_size, 0, "WAL and manifest readahead size");
549 
550 DEFINE_int32(random_access_max_buffer_size, 1024 * 1024,
551              "Maximum windows randomaccess buffer size");
552 
553 DEFINE_int32(writable_file_max_buffer_size, 1024 * 1024,
554              "Maximum write buffer for Writable File");
555 
556 DEFINE_int32(bloom_bits, -1, "Bloom filter bits per key. Negative means"
557              " use default settings.");
558 DEFINE_double(memtable_bloom_size_ratio, 0,
559               "Ratio of memtable size used for bloom filter. 0 means no bloom "
560               "filter.");
561 DEFINE_bool(memtable_whole_key_filtering, false,
562             "Try to use whole key bloom filter in memtables.");
563 DEFINE_bool(memtable_use_huge_page, false,
564             "Try to use huge page in memtables.");
565 
566 DEFINE_bool(use_existing_db, false, "If true, do not destroy the existing"
567             " database.  If you set this flag and also specify a benchmark that"
568             " wants a fresh database, that benchmark will fail.");
569 
570 DEFINE_bool(use_existing_keys, false,
571             "If true, uses existing keys in the DB, "
572             "rather than generating new ones. This involves some startup "
573             "latency to load all keys into memory. It is supported for the "
574             "same read/overwrite benchmarks as `-use_existing_db=true`, which "
575             "must also be set for this flag to be enabled. When this flag is "
576             "set, the value for `-num` will be ignored.");
577 
578 DEFINE_bool(show_table_properties, false,
579             "If true, then per-level table"
580             " properties will be printed on every stats-interval when"
581             " stats_interval is set and stats_per_interval is on.");
582 
583 DEFINE_string(db, "", "Use the db with the following name.");
584 
585 // Read cache flags
586 
587 DEFINE_string(read_cache_path, "",
588               "If not empty string, a read cache will be used in this path");
589 
590 DEFINE_int64(read_cache_size, 4LL * 1024 * 1024 * 1024,
591              "Maximum size of the read cache");
592 
593 DEFINE_bool(read_cache_direct_write, true,
594             "Whether to use Direct IO for writing to the read cache");
595 
596 DEFINE_bool(read_cache_direct_read, true,
597             "Whether to use Direct IO for reading from read cache");
598 
599 DEFINE_bool(use_keep_filter, false, "Whether to use a noop compaction filter");
600 
ValidateCacheNumshardbits(const char * flagname,int32_t value)601 static bool ValidateCacheNumshardbits(const char* flagname, int32_t value) {
602   if (value >= 20) {
603     fprintf(stderr, "Invalid value for --%s: %d, must be < 20\n",
604             flagname, value);
605     return false;
606   }
607   return true;
608 }
609 
610 DEFINE_bool(verify_checksum, true,
611             "Verify checksum for every block read"
612             " from storage");
613 
614 DEFINE_bool(statistics, false, "Database statistics");
615 DEFINE_int32(stats_level, ROCKSDB_NAMESPACE::StatsLevel::kExceptDetailedTimers,
616              "stats level for statistics");
617 DEFINE_string(statistics_string, "", "Serialized statistics string");
618 static class std::shared_ptr<ROCKSDB_NAMESPACE::Statistics> dbstats;
619 
620 DEFINE_int64(writes, -1, "Number of write operations to do. If negative, do"
621              " --num reads.");
622 
623 DEFINE_bool(finish_after_writes, false, "Write thread terminates after all writes are finished");
624 
625 DEFINE_bool(sync, false, "Sync all writes to disk");
626 
627 DEFINE_bool(use_fsync, false, "If true, issue fsync instead of fdatasync");
628 
629 DEFINE_bool(disable_wal, false, "If true, do not write WAL for write.");
630 
631 DEFINE_string(wal_dir, "", "If not empty, use the given dir for WAL");
632 
633 DEFINE_string(truth_db, "/dev/shm/truth_db/dbbench",
634               "Truth key/values used when using verify");
635 
636 DEFINE_int32(num_levels, 7, "The total number of levels");
637 
638 DEFINE_int64(target_file_size_base,
639              ROCKSDB_NAMESPACE::Options().target_file_size_base,
640              "Target file size at level-1");
641 
642 DEFINE_int32(target_file_size_multiplier,
643              ROCKSDB_NAMESPACE::Options().target_file_size_multiplier,
644              "A multiplier to compute target level-N file size (N >= 2)");
645 
646 DEFINE_uint64(max_bytes_for_level_base,
647               ROCKSDB_NAMESPACE::Options().max_bytes_for_level_base,
648               "Max bytes for level-1");
649 
650 DEFINE_bool(level_compaction_dynamic_level_bytes, false,
651             "Whether level size base is dynamic");
652 
653 DEFINE_double(max_bytes_for_level_multiplier, 10,
654               "A multiplier to compute max bytes for level-N (N >= 2)");
655 
656 static std::vector<int> FLAGS_max_bytes_for_level_multiplier_additional_v;
657 DEFINE_string(max_bytes_for_level_multiplier_additional, "",
658               "A vector that specifies additional fanout per level");
659 
660 DEFINE_int32(level0_stop_writes_trigger,
661              ROCKSDB_NAMESPACE::Options().level0_stop_writes_trigger,
662              "Number of files in level-0"
663              " that will trigger put stop.");
664 
665 DEFINE_int32(level0_slowdown_writes_trigger,
666              ROCKSDB_NAMESPACE::Options().level0_slowdown_writes_trigger,
667              "Number of files in level-0"
668              " that will slow down writes.");
669 
670 DEFINE_int32(level0_file_num_compaction_trigger,
671              ROCKSDB_NAMESPACE::Options().level0_file_num_compaction_trigger,
672              "Number of files in level-0"
673              " when compactions start");
674 
ValidateInt32Percent(const char * flagname,int32_t value)675 static bool ValidateInt32Percent(const char* flagname, int32_t value) {
676   if (value <= 0 || value>=100) {
677     fprintf(stderr, "Invalid value for --%s: %d, 0< pct <100 \n",
678             flagname, value);
679     return false;
680   }
681   return true;
682 }
683 DEFINE_int32(readwritepercent, 90, "Ratio of reads to reads/writes (expressed"
684              " as percentage) for the ReadRandomWriteRandom workload. The "
685              "default value 90 means 90% operations out of all reads and writes"
686              " operations are reads. In other words, 9 gets for every 1 put.");
687 
688 DEFINE_int32(mergereadpercent, 70, "Ratio of merges to merges&reads (expressed"
689              " as percentage) for the ReadRandomMergeRandom workload. The"
690              " default value 70 means 70% out of all read and merge operations"
691              " are merges. In other words, 7 merges for every 3 gets.");
692 
693 DEFINE_int32(deletepercent, 2, "Percentage of deletes out of reads/writes/"
694              "deletes (used in RandomWithVerify only). RandomWithVerify "
695              "calculates writepercent as (100 - FLAGS_readwritepercent - "
696              "deletepercent), so deletepercent must be smaller than (100 - "
697              "FLAGS_readwritepercent)");
698 
699 DEFINE_bool(optimize_filters_for_hits, false,
700             "Optimizes bloom filters for workloads for most lookups return "
701             "a value. For now this doesn't create bloom filters for the max "
702             "level of the LSM to reduce metadata that should fit in RAM. ");
703 
704 DEFINE_uint64(delete_obsolete_files_period_micros, 0,
705               "Ignored. Left here for backward compatibility");
706 
707 DEFINE_int64(writes_before_delete_range, 0,
708              "Number of writes before DeleteRange is called regularly.");
709 
710 DEFINE_int64(writes_per_range_tombstone, 0,
711              "Number of writes between range tombstones");
712 
713 DEFINE_int64(range_tombstone_width, 100, "Number of keys in tombstone's range");
714 
715 DEFINE_int64(max_num_range_tombstones, 0,
716              "Maximum number of range tombstones "
717              "to insert.");
718 
719 DEFINE_bool(expand_range_tombstones, false,
720             "Expand range tombstone into sequential regular tombstones.");
721 
722 #ifndef ROCKSDB_LITE
723 // Transactions Options
724 DEFINE_bool(optimistic_transaction_db, false,
725             "Open a OptimisticTransactionDB instance. "
726             "Required for randomtransaction benchmark.");
727 
728 DEFINE_bool(transaction_db, false,
729             "Open a TransactionDB instance. "
730             "Required for randomtransaction benchmark.");
731 
732 DEFINE_uint64(transaction_sets, 2,
733               "Number of keys each transaction will "
734               "modify (use in RandomTransaction only).  Max: 9999");
735 
736 DEFINE_bool(transaction_set_snapshot, false,
737             "Setting to true will have each transaction call SetSnapshot()"
738             " upon creation.");
739 
740 DEFINE_int32(transaction_sleep, 0,
741              "Max microseconds to sleep in between "
742              "reading and writing a value (used in RandomTransaction only). ");
743 
744 DEFINE_uint64(transaction_lock_timeout, 100,
745               "If using a transaction_db, specifies the lock wait timeout in"
746               " milliseconds before failing a transaction waiting on a lock");
747 DEFINE_string(
748     options_file, "",
749     "The path to a RocksDB options file.  If specified, then db_bench will "
750     "run with the RocksDB options in the default column family of the "
751     "specified options file. "
752     "Note that with this setting, db_bench will ONLY accept the following "
753     "RocksDB options related command-line arguments, all other arguments "
754     "that are related to RocksDB options will be ignored:\n"
755     "\t--use_existing_db\n"
756     "\t--use_existing_keys\n"
757     "\t--statistics\n"
758     "\t--row_cache_size\n"
759     "\t--row_cache_numshardbits\n"
760     "\t--enable_io_prio\n"
761     "\t--dump_malloc_stats\n"
762     "\t--num_multi_db\n");
763 
764 // FIFO Compaction Options
765 DEFINE_uint64(fifo_compaction_max_table_files_size_mb, 0,
766               "The limit of total table file sizes to trigger FIFO compaction");
767 
768 DEFINE_bool(fifo_compaction_allow_compaction, true,
769             "Allow compaction in FIFO compaction.");
770 
771 DEFINE_uint64(fifo_compaction_ttl, 0, "TTL for the SST Files in seconds.");
772 
773 // Blob DB Options
774 DEFINE_bool(use_blob_db, false,
775             "Open a BlobDB instance. "
776             "Required for large value benchmark.");
777 
778 DEFINE_bool(
779     blob_db_enable_gc,
780     ROCKSDB_NAMESPACE::blob_db::BlobDBOptions().enable_garbage_collection,
781     "Enable BlobDB garbage collection.");
782 
783 DEFINE_double(
784     blob_db_gc_cutoff,
785     ROCKSDB_NAMESPACE::blob_db::BlobDBOptions().garbage_collection_cutoff,
786     "Cutoff ratio for BlobDB garbage collection.");
787 
788 DEFINE_bool(blob_db_is_fifo,
789             ROCKSDB_NAMESPACE::blob_db::BlobDBOptions().is_fifo,
790             "Enable FIFO eviction strategy in BlobDB.");
791 
792 DEFINE_uint64(blob_db_max_db_size,
793               ROCKSDB_NAMESPACE::blob_db::BlobDBOptions().max_db_size,
794               "Max size limit of the directory where blob files are stored.");
795 
796 DEFINE_uint64(
797     blob_db_max_ttl_range, 0,
798     "TTL range to generate BlobDB data (in seconds). 0 means no TTL.");
799 
800 DEFINE_uint64(blob_db_ttl_range_secs,
801               ROCKSDB_NAMESPACE::blob_db::BlobDBOptions().ttl_range_secs,
802               "TTL bucket size to use when creating blob files.");
803 
804 DEFINE_uint64(blob_db_min_blob_size,
805               ROCKSDB_NAMESPACE::blob_db::BlobDBOptions().min_blob_size,
806               "Smallest blob to store in a file. Blobs smaller than this "
807               "will be inlined with the key in the LSM tree.");
808 
809 DEFINE_uint64(blob_db_bytes_per_sync,
810               ROCKSDB_NAMESPACE::blob_db::BlobDBOptions().bytes_per_sync,
811               "Bytes to sync blob file at.");
812 
813 DEFINE_uint64(blob_db_file_size,
814               ROCKSDB_NAMESPACE::blob_db::BlobDBOptions().blob_file_size,
815               "Target size of each blob file.");
816 
817 DEFINE_string(blob_db_compression_type, "snappy",
818               "Algorithm to use to compress blob in blob file");
819 static enum ROCKSDB_NAMESPACE::CompressionType
820     FLAGS_blob_db_compression_type_e = ROCKSDB_NAMESPACE::kSnappyCompression;
821 
822 // Secondary DB instance Options
823 DEFINE_bool(use_secondary_db, false,
824             "Open a RocksDB secondary instance. A primary instance can be "
825             "running in another db_bench process.");
826 
827 DEFINE_string(secondary_path, "",
828               "Path to a directory used by the secondary instance to store "
829               "private files, e.g. info log.");
830 
831 DEFINE_int32(secondary_update_interval, 5,
832              "Secondary instance attempts to catch up with the primary every "
833              "secondary_update_interval seconds.");
834 
835 #endif  // ROCKSDB_LITE
836 
837 DEFINE_bool(report_bg_io_stats, false,
838             "Measure times spents on I/Os while in compactions. ");
839 
840 DEFINE_bool(use_stderr_info_logger, false,
841             "Write info logs to stderr instead of to LOG file. ");
842 
843 DEFINE_string(trace_file, "", "Trace workload to a file. ");
844 
845 DEFINE_int32(trace_replay_fast_forward, 1,
846              "Fast forward trace replay, must >= 1. ");
847 DEFINE_int32(block_cache_trace_sampling_frequency, 1,
848              "Block cache trace sampling frequency, termed s. It uses spatial "
849              "downsampling and samples accesses to one out of s blocks.");
850 DEFINE_int64(
851     block_cache_trace_max_trace_file_size_in_bytes,
852     uint64_t{64} * 1024 * 1024 * 1024,
853     "The maximum block cache trace file size in bytes. Block cache accesses "
854     "will not be logged if the trace file size exceeds this threshold. Default "
855     "is 64 GB.");
856 DEFINE_string(block_cache_trace_file, "", "Block cache trace file path.");
857 DEFINE_int32(trace_replay_threads, 1,
858              "The number of threads to replay, must >=1.");
859 
StringToCompressionType(const char * ctype)860 static enum ROCKSDB_NAMESPACE::CompressionType StringToCompressionType(
861     const char* ctype) {
862   assert(ctype);
863 
864   if (!strcasecmp(ctype, "none"))
865     return ROCKSDB_NAMESPACE::kNoCompression;
866   else if (!strcasecmp(ctype, "snappy"))
867     return ROCKSDB_NAMESPACE::kSnappyCompression;
868   else if (!strcasecmp(ctype, "zlib"))
869     return ROCKSDB_NAMESPACE::kZlibCompression;
870   else if (!strcasecmp(ctype, "bzip2"))
871     return ROCKSDB_NAMESPACE::kBZip2Compression;
872   else if (!strcasecmp(ctype, "lz4"))
873     return ROCKSDB_NAMESPACE::kLZ4Compression;
874   else if (!strcasecmp(ctype, "lz4hc"))
875     return ROCKSDB_NAMESPACE::kLZ4HCCompression;
876   else if (!strcasecmp(ctype, "xpress"))
877     return ROCKSDB_NAMESPACE::kXpressCompression;
878   else if (!strcasecmp(ctype, "zstd"))
879     return ROCKSDB_NAMESPACE::kZSTD;
880 
881   fprintf(stdout, "Cannot parse compression type '%s'\n", ctype);
882   return ROCKSDB_NAMESPACE::kSnappyCompression;  // default value
883 }
884 
ColumnFamilyName(size_t i)885 static std::string ColumnFamilyName(size_t i) {
886   if (i == 0) {
887     return ROCKSDB_NAMESPACE::kDefaultColumnFamilyName;
888   } else {
889     char name[100];
890     snprintf(name, sizeof(name), "column_family_name_%06zu", i);
891     return std::string(name);
892   }
893 }
894 
895 DEFINE_string(compression_type, "snappy",
896               "Algorithm to use to compress the database");
897 static enum ROCKSDB_NAMESPACE::CompressionType FLAGS_compression_type_e =
898     ROCKSDB_NAMESPACE::kSnappyCompression;
899 
900 DEFINE_int64(sample_for_compression, 0, "Sample every N block for compression");
901 
902 DEFINE_int32(compression_level, ROCKSDB_NAMESPACE::CompressionOptions().level,
903              "Compression level. The meaning of this value is library-"
904              "dependent. If unset, we try to use the default for the library "
905              "specified in `--compression_type`");
906 
907 DEFINE_int32(compression_max_dict_bytes,
908              ROCKSDB_NAMESPACE::CompressionOptions().max_dict_bytes,
909              "Maximum size of dictionary used to prime the compression "
910              "library.");
911 
912 DEFINE_int32(compression_zstd_max_train_bytes,
913              ROCKSDB_NAMESPACE::CompressionOptions().zstd_max_train_bytes,
914              "Maximum size of training data passed to zstd's dictionary "
915              "trainer.");
916 
917 DEFINE_int32(min_level_to_compress, -1, "If non-negative, compression starts"
918              " from this level. Levels with number < min_level_to_compress are"
919              " not compressed. Otherwise, apply compression_type to "
920              "all levels.");
921 
ValidateTableCacheNumshardbits(const char * flagname,int32_t value)922 static bool ValidateTableCacheNumshardbits(const char* flagname,
923                                            int32_t value) {
924   if (0 >= value || value > 20) {
925     fprintf(stderr, "Invalid value for --%s: %d, must be  0 < val <= 20\n",
926             flagname, value);
927     return false;
928   }
929   return true;
930 }
931 DEFINE_int32(table_cache_numshardbits, 4, "");
932 
933 #ifndef ROCKSDB_LITE
934 DEFINE_string(env_uri, "", "URI for registry Env lookup. Mutually exclusive"
935               " with --hdfs.");
936 #endif  // ROCKSDB_LITE
937 DEFINE_string(hdfs, "", "Name of hdfs environment. Mutually exclusive with"
938               " --env_uri.");
939 
940 static std::shared_ptr<ROCKSDB_NAMESPACE::Env> env_guard;
941 
942 static ROCKSDB_NAMESPACE::Env* FLAGS_env = ROCKSDB_NAMESPACE::Env::Default();
943 
944 DEFINE_int64(stats_interval, 0, "Stats are reported every N operations when "
945              "this is greater than zero. When 0 the interval grows over time.");
946 
947 DEFINE_int64(stats_interval_seconds, 0, "Report stats every N seconds. This "
948              "overrides stats_interval when both are > 0.");
949 
950 DEFINE_int32(stats_per_interval, 0, "Reports additional stats per interval when"
951              " this is greater than 0.");
952 
953 DEFINE_int64(report_interval_seconds, 0,
954              "If greater than zero, it will write simple stats in CVS format "
955              "to --report_file every N seconds");
956 
957 DEFINE_string(report_file, "report.csv",
958               "Filename where some simple stats are reported to (if "
959               "--report_interval_seconds is bigger than 0)");
960 
961 DEFINE_int32(thread_status_per_interval, 0,
962              "Takes and report a snapshot of the current status of each thread"
963              " when this is greater than 0.");
964 
965 DEFINE_int32(perf_level, ROCKSDB_NAMESPACE::PerfLevel::kDisable,
966              "Level of perf collection");
967 
ValidateRateLimit(const char * flagname,double value)968 static bool ValidateRateLimit(const char* flagname, double value) {
969   const double EPSILON = 1e-10;
970   if ( value < -EPSILON ) {
971     fprintf(stderr, "Invalid value for --%s: %12.6f, must be >= 0.0\n",
972             flagname, value);
973     return false;
974   }
975   return true;
976 }
977 DEFINE_double(soft_rate_limit, 0.0, "DEPRECATED");
978 
979 DEFINE_double(hard_rate_limit, 0.0, "DEPRECATED");
980 
981 DEFINE_uint64(soft_pending_compaction_bytes_limit, 64ull * 1024 * 1024 * 1024,
982               "Slowdown writes if pending compaction bytes exceed this number");
983 
984 DEFINE_uint64(hard_pending_compaction_bytes_limit, 128ull * 1024 * 1024 * 1024,
985               "Stop writes if pending compaction bytes exceed this number");
986 
987 DEFINE_uint64(delayed_write_rate, 8388608u,
988               "Limited bytes allowed to DB when soft_rate_limit or "
989               "level0_slowdown_writes_trigger triggers");
990 
991 DEFINE_bool(enable_pipelined_write, true,
992             "Allow WAL and memtable writes to be pipelined");
993 
994 DEFINE_bool(
995     unordered_write, false,
996     "Enable the unordered write feature, which provides higher throughput but "
997     "relaxes the guarantees around atomic reads and immutable snapshots");
998 
999 DEFINE_bool(allow_concurrent_memtable_write, true,
1000             "Allow multi-writers to update mem tables in parallel.");
1001 
1002 DEFINE_bool(inplace_update_support,
1003             ROCKSDB_NAMESPACE::Options().inplace_update_support,
1004             "Support in-place memtable update for smaller or same-size values");
1005 
1006 DEFINE_uint64(inplace_update_num_locks,
1007               ROCKSDB_NAMESPACE::Options().inplace_update_num_locks,
1008               "Number of RW locks to protect in-place memtable updates");
1009 
1010 DEFINE_bool(enable_write_thread_adaptive_yield, true,
1011             "Use a yielding spin loop for brief writer thread waits.");
1012 
1013 DEFINE_uint64(
1014     write_thread_max_yield_usec, 100,
1015     "Maximum microseconds for enable_write_thread_adaptive_yield operation.");
1016 
1017 DEFINE_uint64(write_thread_slow_yield_usec, 3,
1018               "The threshold at which a slow yield is considered a signal that "
1019               "other processes or threads want the core.");
1020 
1021 DEFINE_int32(rate_limit_delay_max_milliseconds, 1000,
1022              "When hard_rate_limit is set then this is the max time a put will"
1023              " be stalled.");
1024 
1025 DEFINE_uint64(rate_limiter_bytes_per_sec, 0, "Set options.rate_limiter value.");
1026 
1027 DEFINE_bool(rate_limiter_auto_tuned, false,
1028             "Enable dynamic adjustment of rate limit according to demand for "
1029             "background I/O");
1030 
1031 
1032 DEFINE_bool(sine_write_rate, false,
1033             "Use a sine wave write_rate_limit");
1034 
1035 DEFINE_uint64(sine_write_rate_interval_milliseconds, 10000,
1036               "Interval of which the sine wave write_rate_limit is recalculated");
1037 
1038 DEFINE_double(sine_a, 1,
1039              "A in f(x) = A sin(bx + c) + d");
1040 
1041 DEFINE_double(sine_b, 1,
1042              "B in f(x) = A sin(bx + c) + d");
1043 
1044 DEFINE_double(sine_c, 0,
1045              "C in f(x) = A sin(bx + c) + d");
1046 
1047 DEFINE_double(sine_d, 1,
1048              "D in f(x) = A sin(bx + c) + d");
1049 
1050 DEFINE_bool(rate_limit_bg_reads, false,
1051             "Use options.rate_limiter on compaction reads");
1052 
1053 DEFINE_uint64(
1054     benchmark_write_rate_limit, 0,
1055     "If non-zero, db_bench will rate-limit the writes going into RocksDB. This "
1056     "is the global rate in bytes/second.");
1057 
1058 // the parameters of mix_graph
1059 DEFINE_double(keyrange_dist_a, 0.0,
1060               "The parameter 'a' of prefix average access distribution "
1061               "f(x)=a*exp(b*x)+c*exp(d*x)");
1062 DEFINE_double(keyrange_dist_b, 0.0,
1063               "The parameter 'b' of prefix average access distribution "
1064               "f(x)=a*exp(b*x)+c*exp(d*x)");
1065 DEFINE_double(keyrange_dist_c, 0.0,
1066               "The parameter 'c' of prefix average access distribution"
1067               "f(x)=a*exp(b*x)+c*exp(d*x)");
1068 DEFINE_double(keyrange_dist_d, 0.0,
1069               "The parameter 'd' of prefix average access distribution"
1070               "f(x)=a*exp(b*x)+c*exp(d*x)");
1071 DEFINE_int64(keyrange_num, 1,
1072              "The number of key ranges that are in the same prefix "
1073              "group, each prefix range will have its key access "
1074              "distribution");
1075 DEFINE_double(key_dist_a, 0.0,
1076               "The parameter 'a' of key access distribution model "
1077               "f(x)=a*x^b");
1078 DEFINE_double(key_dist_b, 0.0,
1079               "The parameter 'b' of key access distribution model "
1080               "f(x)=a*x^b");
1081 DEFINE_double(value_theta, 0.0,
1082               "The parameter 'theta' of Generized Pareto Distribution "
1083               "f(x)=(1/sigma)*(1+k*(x-theta)/sigma)^-(1/k+1)");
1084 DEFINE_double(value_k, 0.0,
1085               "The parameter 'k' of Generized Pareto Distribution "
1086               "f(x)=(1/sigma)*(1+k*(x-theta)/sigma)^-(1/k+1)");
1087 DEFINE_double(value_sigma, 0.0,
1088               "The parameter 'theta' of Generized Pareto Distribution "
1089               "f(x)=(1/sigma)*(1+k*(x-theta)/sigma)^-(1/k+1)");
1090 DEFINE_double(iter_theta, 0.0,
1091               "The parameter 'theta' of Generized Pareto Distribution "
1092               "f(x)=(1/sigma)*(1+k*(x-theta)/sigma)^-(1/k+1)");
1093 DEFINE_double(iter_k, 0.0,
1094               "The parameter 'k' of Generized Pareto Distribution "
1095               "f(x)=(1/sigma)*(1+k*(x-theta)/sigma)^-(1/k+1)");
1096 DEFINE_double(iter_sigma, 0.0,
1097               "The parameter 'sigma' of Generized Pareto Distribution "
1098               "f(x)=(1/sigma)*(1+k*(x-theta)/sigma)^-(1/k+1)");
1099 DEFINE_double(mix_get_ratio, 1.0,
1100               "The ratio of Get queries of mix_graph workload");
1101 DEFINE_double(mix_put_ratio, 0.0,
1102               "The ratio of Put queries of mix_graph workload");
1103 DEFINE_double(mix_seek_ratio, 0.0,
1104               "The ratio of Seek queries of mix_graph workload");
1105 DEFINE_int64(mix_max_scan_len, 10000, "The max scan length of Iterator");
1106 DEFINE_int64(mix_ave_kv_size, 512,
1107              "The average key-value size of this workload");
1108 DEFINE_int64(mix_max_value_size, 1024, "The max value size of this workload");
1109 DEFINE_double(
1110     sine_mix_rate_noise, 0.0,
1111     "Add the noise ratio to the sine rate, it is between 0.0 and 1.0");
1112 DEFINE_bool(sine_mix_rate, false,
1113             "Enable the sine QPS control on the mix workload");
1114 DEFINE_uint64(
1115     sine_mix_rate_interval_milliseconds, 10000,
1116     "Interval of which the sine wave read_rate_limit is recalculated");
1117 DEFINE_int64(mix_accesses, -1,
1118              "The total query accesses of mix_graph workload");
1119 
1120 DEFINE_uint64(
1121     benchmark_read_rate_limit, 0,
1122     "If non-zero, db_bench will rate-limit the reads from RocksDB. This "
1123     "is the global rate in ops/second.");
1124 
1125 DEFINE_uint64(max_compaction_bytes,
1126               ROCKSDB_NAMESPACE::Options().max_compaction_bytes,
1127               "Max bytes allowed in one compaction");
1128 
1129 #ifndef ROCKSDB_LITE
1130 DEFINE_bool(readonly, false, "Run read only benchmarks.");
1131 
1132 DEFINE_bool(print_malloc_stats, false,
1133             "Print malloc stats to stdout after benchmarks finish.");
1134 #endif  // ROCKSDB_LITE
1135 
1136 DEFINE_bool(disable_auto_compactions, false, "Do not auto trigger compactions");
1137 
1138 DEFINE_uint64(wal_ttl_seconds, 0, "Set the TTL for the WAL Files in seconds.");
1139 DEFINE_uint64(wal_size_limit_MB, 0, "Set the size limit for the WAL Files"
1140               " in MB.");
1141 DEFINE_uint64(max_total_wal_size, 0, "Set total max WAL size");
1142 
1143 DEFINE_bool(mmap_read, ROCKSDB_NAMESPACE::Options().allow_mmap_reads,
1144             "Allow reads to occur via mmap-ing files");
1145 
1146 DEFINE_bool(mmap_write, ROCKSDB_NAMESPACE::Options().allow_mmap_writes,
1147             "Allow writes to occur via mmap-ing files");
1148 
1149 DEFINE_bool(use_direct_reads, ROCKSDB_NAMESPACE::Options().use_direct_reads,
1150             "Use O_DIRECT for reading data");
1151 
1152 DEFINE_bool(use_direct_io_for_flush_and_compaction,
1153             ROCKSDB_NAMESPACE::Options().use_direct_io_for_flush_and_compaction,
1154             "Use O_DIRECT for background flush and compaction writes");
1155 
1156 DEFINE_bool(advise_random_on_open,
1157             ROCKSDB_NAMESPACE::Options().advise_random_on_open,
1158             "Advise random access on table file open");
1159 
1160 DEFINE_string(compaction_fadvice, "NORMAL",
1161               "Access pattern advice when a file is compacted");
1162 static auto FLAGS_compaction_fadvice_e =
1163     ROCKSDB_NAMESPACE::Options().access_hint_on_compaction_start;
1164 
1165 DEFINE_bool(use_tailing_iterator, false,
1166             "Use tailing iterator to access a series of keys instead of get");
1167 
1168 DEFINE_bool(use_adaptive_mutex, ROCKSDB_NAMESPACE::Options().use_adaptive_mutex,
1169             "Use adaptive mutex");
1170 
1171 DEFINE_uint64(bytes_per_sync, ROCKSDB_NAMESPACE::Options().bytes_per_sync,
1172               "Allows OS to incrementally sync SST files to disk while they are"
1173               " being written, in the background. Issue one request for every"
1174               " bytes_per_sync written. 0 turns it off.");
1175 
1176 DEFINE_uint64(wal_bytes_per_sync,
1177               ROCKSDB_NAMESPACE::Options().wal_bytes_per_sync,
1178               "Allows OS to incrementally sync WAL files to disk while they are"
1179               " being written, in the background. Issue one request for every"
1180               " wal_bytes_per_sync written. 0 turns it off.");
1181 
1182 DEFINE_bool(use_single_deletes, true,
1183             "Use single deletes (used in RandomReplaceKeys only).");
1184 
1185 DEFINE_double(stddev, 2000.0,
1186               "Standard deviation of normal distribution used for picking keys"
1187               " (used in RandomReplaceKeys only).");
1188 
1189 DEFINE_int32(key_id_range, 100000,
1190              "Range of possible value of key id (used in TimeSeries only).");
1191 
1192 DEFINE_string(expire_style, "none",
1193               "Style to remove expired time entries. Can be one of the options "
1194               "below: none (do not expired data), compaction_filter (use a "
1195               "compaction filter to remove expired data), delete (seek IDs and "
1196               "remove expired data) (used in TimeSeries only).");
1197 
1198 DEFINE_uint64(
1199     time_range, 100000,
1200     "Range of timestamp that store in the database (used in TimeSeries"
1201     " only).");
1202 
1203 DEFINE_int32(num_deletion_threads, 1,
1204              "Number of threads to do deletion (used in TimeSeries and delete "
1205              "expire_style only).");
1206 
1207 DEFINE_int32(max_successive_merges, 0, "Maximum number of successive merge"
1208              " operations on a key in the memtable");
1209 
ValidatePrefixSize(const char * flagname,int32_t value)1210 static bool ValidatePrefixSize(const char* flagname, int32_t value) {
1211   if (value < 0 || value>=2000000000) {
1212     fprintf(stderr, "Invalid value for --%s: %d. 0<= PrefixSize <=2000000000\n",
1213             flagname, value);
1214     return false;
1215   }
1216   return true;
1217 }
1218 
1219 DEFINE_int32(prefix_size, 0, "control the prefix size for HashSkipList and "
1220              "plain table");
1221 DEFINE_int64(keys_per_prefix, 0, "control average number of keys generated "
1222              "per prefix, 0 means no special handling of the prefix, "
1223              "i.e. use the prefix comes with the generated random number.");
1224 DEFINE_bool(total_order_seek, false,
1225             "Enable total order seek regardless of index format.");
1226 DEFINE_bool(prefix_same_as_start, false,
1227             "Enforce iterator to return keys with prefix same as seek key.");
1228 DEFINE_bool(
1229     seek_missing_prefix, false,
1230     "Iterator seek to keys with non-exist prefixes. Require prefix_size > 8");
1231 
1232 DEFINE_int32(memtable_insert_with_hint_prefix_size, 0,
1233              "If non-zero, enable "
1234              "memtable insert with hint with the given prefix size.");
1235 DEFINE_bool(enable_io_prio, false, "Lower the background flush/compaction "
1236             "threads' IO priority");
1237 DEFINE_bool(enable_cpu_prio, false, "Lower the background flush/compaction "
1238             "threads' CPU priority");
1239 DEFINE_bool(identity_as_first_hash, false, "the first hash function of cuckoo "
1240             "table becomes an identity function. This is only valid when key "
1241             "is 8 bytes");
1242 DEFINE_bool(dump_malloc_stats, true, "Dump malloc stats in LOG ");
1243 DEFINE_uint64(stats_dump_period_sec,
1244               ROCKSDB_NAMESPACE::Options().stats_dump_period_sec,
1245               "Gap between printing stats to log in seconds");
1246 DEFINE_uint64(stats_persist_period_sec,
1247               ROCKSDB_NAMESPACE::Options().stats_persist_period_sec,
1248               "Gap between persisting stats in seconds");
1249 DEFINE_bool(persist_stats_to_disk,
1250             ROCKSDB_NAMESPACE::Options().persist_stats_to_disk,
1251             "whether to persist stats to disk");
1252 DEFINE_uint64(stats_history_buffer_size,
1253               ROCKSDB_NAMESPACE::Options().stats_history_buffer_size,
1254               "Max number of stats snapshots to keep in memory");
1255 DEFINE_int64(multiread_stride, 0,
1256              "Stride length for the keys in a MultiGet batch");
1257 DEFINE_bool(multiread_batched, false, "Use the new MultiGet API");
1258 
1259 enum RepFactory {
1260   kSkipList,
1261   kPrefixHash,
1262   kVectorRep,
1263   kHashLinkedList,
1264 };
1265 
StringToRepFactory(const char * ctype)1266 static enum RepFactory StringToRepFactory(const char* ctype) {
1267   assert(ctype);
1268 
1269   if (!strcasecmp(ctype, "skip_list"))
1270     return kSkipList;
1271   else if (!strcasecmp(ctype, "prefix_hash"))
1272     return kPrefixHash;
1273   else if (!strcasecmp(ctype, "vector"))
1274     return kVectorRep;
1275   else if (!strcasecmp(ctype, "hash_linkedlist"))
1276     return kHashLinkedList;
1277 
1278   fprintf(stdout, "Cannot parse memreptable %s\n", ctype);
1279   return kSkipList;
1280 }
1281 
1282 static enum RepFactory FLAGS_rep_factory;
1283 DEFINE_string(memtablerep, "skip_list", "");
1284 DEFINE_int64(hash_bucket_count, 1024 * 1024, "hash bucket count");
1285 DEFINE_bool(use_plain_table, false, "if use plain table "
1286             "instead of block-based table format");
1287 DEFINE_bool(use_cuckoo_table, false, "if use cuckoo table format");
1288 DEFINE_double(cuckoo_hash_ratio, 0.9, "Hash ratio for Cuckoo SST table.");
1289 DEFINE_bool(use_hash_search, false, "if use kHashSearch "
1290             "instead of kBinarySearch. "
1291             "This is valid if only we use BlockTable");
1292 DEFINE_bool(use_block_based_filter, false, "if use kBlockBasedFilter "
1293             "instead of kFullFilter for filter block. "
1294             "This is valid if only we use BlockTable");
1295 DEFINE_string(merge_operator, "", "The merge operator to use with the database."
1296               "If a new merge operator is specified, be sure to use fresh"
1297               " database The possible merge operators are defined in"
1298               " utilities/merge_operators.h");
1299 DEFINE_int32(skip_list_lookahead, 0, "Used with skip_list memtablerep; try "
1300              "linear search first for this many steps from the previous "
1301              "position");
1302 DEFINE_bool(report_file_operations, false, "if report number of file "
1303             "operations");
1304 DEFINE_int32(readahead_size, 0, "Iterator readahead size");
1305 
1306 static const bool FLAGS_soft_rate_limit_dummy __attribute__((__unused__)) =
1307     RegisterFlagValidator(&FLAGS_soft_rate_limit, &ValidateRateLimit);
1308 
1309 static const bool FLAGS_hard_rate_limit_dummy __attribute__((__unused__)) =
1310     RegisterFlagValidator(&FLAGS_hard_rate_limit, &ValidateRateLimit);
1311 
1312 static const bool FLAGS_prefix_size_dummy __attribute__((__unused__)) =
1313     RegisterFlagValidator(&FLAGS_prefix_size, &ValidatePrefixSize);
1314 
1315 static const bool FLAGS_key_size_dummy __attribute__((__unused__)) =
1316     RegisterFlagValidator(&FLAGS_key_size, &ValidateKeySize);
1317 
1318 static const bool FLAGS_cache_numshardbits_dummy __attribute__((__unused__)) =
1319     RegisterFlagValidator(&FLAGS_cache_numshardbits,
1320                           &ValidateCacheNumshardbits);
1321 
1322 static const bool FLAGS_readwritepercent_dummy __attribute__((__unused__)) =
1323     RegisterFlagValidator(&FLAGS_readwritepercent, &ValidateInt32Percent);
1324 
1325 DEFINE_int32(disable_seek_compaction, false,
1326              "Not used, left here for backwards compatibility");
1327 
1328 static const bool FLAGS_deletepercent_dummy __attribute__((__unused__)) =
1329     RegisterFlagValidator(&FLAGS_deletepercent, &ValidateInt32Percent);
1330 static const bool FLAGS_table_cache_numshardbits_dummy __attribute__((__unused__)) =
1331     RegisterFlagValidator(&FLAGS_table_cache_numshardbits,
1332                           &ValidateTableCacheNumshardbits);
1333 
1334 namespace ROCKSDB_NAMESPACE {
1335 
1336 namespace {
1337 struct ReportFileOpCounters {
1338   std::atomic<int> open_counter_;
1339   std::atomic<int> read_counter_;
1340   std::atomic<int> append_counter_;
1341   std::atomic<uint64_t> bytes_read_;
1342   std::atomic<uint64_t> bytes_written_;
1343 };
1344 
1345 // A special Env to records and report file operations in db_bench
1346 class ReportFileOpEnv : public EnvWrapper {
1347  public:
ReportFileOpEnv(Env * base)1348   explicit ReportFileOpEnv(Env* base) : EnvWrapper(base) { reset(); }
1349 
reset()1350   void reset() {
1351     counters_.open_counter_ = 0;
1352     counters_.read_counter_ = 0;
1353     counters_.append_counter_ = 0;
1354     counters_.bytes_read_ = 0;
1355     counters_.bytes_written_ = 0;
1356   }
1357 
NewSequentialFile(const std::string & f,std::unique_ptr<SequentialFile> * r,const EnvOptions & soptions)1358   Status NewSequentialFile(const std::string& f,
1359                            std::unique_ptr<SequentialFile>* r,
1360                            const EnvOptions& soptions) override {
1361     class CountingFile : public SequentialFile {
1362      private:
1363       std::unique_ptr<SequentialFile> target_;
1364       ReportFileOpCounters* counters_;
1365 
1366      public:
1367       CountingFile(std::unique_ptr<SequentialFile>&& target,
1368                    ReportFileOpCounters* counters)
1369           : target_(std::move(target)), counters_(counters) {}
1370 
1371       Status Read(size_t n, Slice* result, char* scratch) override {
1372         counters_->read_counter_.fetch_add(1, std::memory_order_relaxed);
1373         Status rv = target_->Read(n, result, scratch);
1374         counters_->bytes_read_.fetch_add(result->size(),
1375                                          std::memory_order_relaxed);
1376         return rv;
1377       }
1378 
1379       Status Skip(uint64_t n) override { return target_->Skip(n); }
1380     };
1381 
1382     Status s = target()->NewSequentialFile(f, r, soptions);
1383     if (s.ok()) {
1384       counters()->open_counter_.fetch_add(1, std::memory_order_relaxed);
1385       r->reset(new CountingFile(std::move(*r), counters()));
1386     }
1387     return s;
1388   }
1389 
NewRandomAccessFile(const std::string & f,std::unique_ptr<RandomAccessFile> * r,const EnvOptions & soptions)1390   Status NewRandomAccessFile(const std::string& f,
1391                              std::unique_ptr<RandomAccessFile>* r,
1392                              const EnvOptions& soptions) override {
1393     class CountingFile : public RandomAccessFile {
1394      private:
1395       std::unique_ptr<RandomAccessFile> target_;
1396       ReportFileOpCounters* counters_;
1397 
1398      public:
1399       CountingFile(std::unique_ptr<RandomAccessFile>&& target,
1400                    ReportFileOpCounters* counters)
1401           : target_(std::move(target)), counters_(counters) {}
1402       Status Read(uint64_t offset, size_t n, Slice* result,
1403                   char* scratch) const override {
1404         counters_->read_counter_.fetch_add(1, std::memory_order_relaxed);
1405         Status rv = target_->Read(offset, n, result, scratch);
1406         counters_->bytes_read_.fetch_add(result->size(),
1407                                          std::memory_order_relaxed);
1408         return rv;
1409       }
1410     };
1411 
1412     Status s = target()->NewRandomAccessFile(f, r, soptions);
1413     if (s.ok()) {
1414       counters()->open_counter_.fetch_add(1, std::memory_order_relaxed);
1415       r->reset(new CountingFile(std::move(*r), counters()));
1416     }
1417     return s;
1418   }
1419 
NewWritableFile(const std::string & f,std::unique_ptr<WritableFile> * r,const EnvOptions & soptions)1420   Status NewWritableFile(const std::string& f, std::unique_ptr<WritableFile>* r,
1421                          const EnvOptions& soptions) override {
1422     class CountingFile : public WritableFile {
1423      private:
1424       std::unique_ptr<WritableFile> target_;
1425       ReportFileOpCounters* counters_;
1426 
1427      public:
1428       CountingFile(std::unique_ptr<WritableFile>&& target,
1429                    ReportFileOpCounters* counters)
1430           : target_(std::move(target)), counters_(counters) {}
1431 
1432       Status Append(const Slice& data) override {
1433         counters_->append_counter_.fetch_add(1, std::memory_order_relaxed);
1434         Status rv = target_->Append(data);
1435         counters_->bytes_written_.fetch_add(data.size(),
1436                                             std::memory_order_relaxed);
1437         return rv;
1438       }
1439 
1440       Status Truncate(uint64_t size) override { return target_->Truncate(size); }
1441       Status Close() override { return target_->Close(); }
1442       Status Flush() override { return target_->Flush(); }
1443       Status Sync() override { return target_->Sync(); }
1444     };
1445 
1446     Status s = target()->NewWritableFile(f, r, soptions);
1447     if (s.ok()) {
1448       counters()->open_counter_.fetch_add(1, std::memory_order_relaxed);
1449       r->reset(new CountingFile(std::move(*r), counters()));
1450     }
1451     return s;
1452   }
1453 
1454   // getter
counters()1455   ReportFileOpCounters* counters() { return &counters_; }
1456 
1457  private:
1458   ReportFileOpCounters counters_;
1459 };
1460 
1461 }  // namespace
1462 
1463 enum DistributionType : unsigned char {
1464   kFixed = 0,
1465   kUniform,
1466   kNormal
1467 };
1468 
1469 static enum DistributionType FLAGS_value_size_distribution_type_e = kFixed;
1470 
StringToDistributionType(const char * ctype)1471 static enum DistributionType StringToDistributionType(const char* ctype) {
1472   assert(ctype);
1473 
1474   if (!strcasecmp(ctype, "fixed"))
1475     return kFixed;
1476   else if (!strcasecmp(ctype, "uniform"))
1477     return kUniform;
1478   else if (!strcasecmp(ctype, "normal"))
1479     return kNormal;
1480 
1481   fprintf(stdout, "Cannot parse distribution type '%s'\n", ctype);
1482   return kFixed;  // default value
1483 }
1484 
1485 class BaseDistribution {
1486  public:
BaseDistribution(unsigned int _min,unsigned int _max)1487   BaseDistribution(unsigned int _min, unsigned int _max)
1488       : min_value_size_(_min), max_value_size_(_max) {}
~BaseDistribution()1489   virtual ~BaseDistribution() {}
1490 
Generate()1491   unsigned int Generate() {
1492     auto val = Get();
1493     if (NeedTruncate()) {
1494       val = std::max(min_value_size_, val);
1495       val = std::min(max_value_size_, val);
1496     }
1497     return val;
1498   }
1499  private:
1500   virtual unsigned int Get() = 0;
NeedTruncate()1501   virtual bool NeedTruncate() {
1502     return true;
1503   }
1504   unsigned int min_value_size_;
1505   unsigned int max_value_size_;
1506 };
1507 
1508 class FixedDistribution : public BaseDistribution
1509 {
1510  public:
FixedDistribution(unsigned int size)1511   FixedDistribution(unsigned int size) :
1512     BaseDistribution(size, size),
1513     size_(size) {}
1514  private:
Get()1515   virtual unsigned int Get() override {
1516     return size_;
1517   }
NeedTruncate()1518   virtual bool NeedTruncate() override {
1519     return false;
1520   }
1521   unsigned int size_;
1522 };
1523 
1524 class NormalDistribution
1525     : public BaseDistribution, public std::normal_distribution<double> {
1526  public:
NormalDistribution(unsigned int _min,unsigned int _max)1527   NormalDistribution(unsigned int _min, unsigned int _max)
1528       : BaseDistribution(_min, _max),
1529         // 99.7% values within the range [min, max].
1530         std::normal_distribution<double>(
1531             (double)(_min + _max) / 2.0 /*mean*/,
1532             (double)(_max - _min) / 6.0 /*stddev*/),
1533         gen_(rd_()) {}
1534 
1535  private:
Get()1536   virtual unsigned int Get() override {
1537     return static_cast<unsigned int>((*this)(gen_));
1538   }
1539   std::random_device rd_;
1540   std::mt19937 gen_;
1541 };
1542 
1543 class UniformDistribution
1544     : public BaseDistribution,
1545       public std::uniform_int_distribution<unsigned int> {
1546  public:
UniformDistribution(unsigned int _min,unsigned int _max)1547   UniformDistribution(unsigned int _min, unsigned int _max)
1548       : BaseDistribution(_min, _max),
1549         std::uniform_int_distribution<unsigned int>(_min, _max),
1550         gen_(rd_()) {}
1551 
1552  private:
Get()1553   virtual unsigned int Get() override {
1554     return (*this)(gen_);
1555   }
NeedTruncate()1556   virtual bool NeedTruncate() override {
1557     return false;
1558   }
1559   std::random_device rd_;
1560   std::mt19937 gen_;
1561 };
1562 
1563 // Helper for quickly generating random data.
1564 class RandomGenerator {
1565  private:
1566   std::string data_;
1567   unsigned int pos_;
1568   std::unique_ptr<BaseDistribution> dist_;
1569 
1570  public:
1571 
RandomGenerator()1572   RandomGenerator() {
1573     auto max_value_size = FLAGS_value_size_max;
1574     switch (FLAGS_value_size_distribution_type_e) {
1575       case kUniform:
1576         dist_.reset(new UniformDistribution(FLAGS_value_size_min,
1577                                             FLAGS_value_size_max));
1578         break;
1579       case kNormal:
1580         dist_.reset(new NormalDistribution(FLAGS_value_size_min,
1581                                            FLAGS_value_size_max));
1582         break;
1583       case kFixed:
1584       default:
1585         dist_.reset(new FixedDistribution(value_size));
1586         max_value_size = value_size;
1587     }
1588     // We use a limited amount of data over and over again and ensure
1589     // that it is larger than the compression window (32KB), and also
1590     // large enough to serve all typical value sizes we want to write.
1591     Random rnd(301);
1592     std::string piece;
1593     while (data_.size() < (unsigned)std::max(1048576, max_value_size)) {
1594       // Add a short fragment that is as compressible as specified
1595       // by FLAGS_compression_ratio.
1596       test::CompressibleString(&rnd, FLAGS_compression_ratio, 100, &piece);
1597       data_.append(piece);
1598     }
1599     pos_ = 0;
1600   }
1601 
Generate(unsigned int len)1602   Slice Generate(unsigned int len) {
1603     assert(len <= data_.size());
1604     if (pos_ + len > data_.size()) {
1605       pos_ = 0;
1606     }
1607     pos_ += len;
1608     return Slice(data_.data() + pos_ - len, len);
1609   }
1610 
Generate()1611   Slice Generate() {
1612     auto len = dist_->Generate();
1613     return Generate(len);
1614   }
1615 };
1616 
AppendWithSpace(std::string * str,Slice msg)1617 static void AppendWithSpace(std::string* str, Slice msg) {
1618   if (msg.empty()) return;
1619   if (!str->empty()) {
1620     str->push_back(' ');
1621   }
1622   str->append(msg.data(), msg.size());
1623 }
1624 
1625 struct DBWithColumnFamilies {
1626   std::vector<ColumnFamilyHandle*> cfh;
1627   DB* db;
1628 #ifndef ROCKSDB_LITE
1629   OptimisticTransactionDB* opt_txn_db;
1630 #endif  // ROCKSDB_LITE
1631   std::atomic<size_t> num_created;  // Need to be updated after all the
1632                                     // new entries in cfh are set.
1633   size_t num_hot;  // Number of column families to be queried at each moment.
1634                    // After each CreateNewCf(), another num_hot number of new
1635                    // Column families will be created and used to be queried.
1636   port::Mutex create_cf_mutex;  // Only one thread can execute CreateNewCf()
1637   std::vector<int> cfh_idx_to_prob;  // ith index holds probability of operating
1638                                      // on cfh[i].
1639 
DBWithColumnFamiliesROCKSDB_NAMESPACE::DBWithColumnFamilies1640   DBWithColumnFamilies()
1641       : db(nullptr)
1642 #ifndef ROCKSDB_LITE
1643         , opt_txn_db(nullptr)
1644 #endif  // ROCKSDB_LITE
1645   {
1646     cfh.clear();
1647     num_created = 0;
1648     num_hot = 0;
1649   }
1650 
DBWithColumnFamiliesROCKSDB_NAMESPACE::DBWithColumnFamilies1651   DBWithColumnFamilies(const DBWithColumnFamilies& other)
1652       : cfh(other.cfh),
1653         db(other.db),
1654 #ifndef ROCKSDB_LITE
1655         opt_txn_db(other.opt_txn_db),
1656 #endif  // ROCKSDB_LITE
1657         num_created(other.num_created.load()),
1658         num_hot(other.num_hot),
1659         cfh_idx_to_prob(other.cfh_idx_to_prob) {
1660   }
1661 
DeleteDBsROCKSDB_NAMESPACE::DBWithColumnFamilies1662   void DeleteDBs() {
1663     std::for_each(cfh.begin(), cfh.end(),
1664                   [](ColumnFamilyHandle* cfhi) { delete cfhi; });
1665     cfh.clear();
1666 #ifndef ROCKSDB_LITE
1667     if (opt_txn_db) {
1668       delete opt_txn_db;
1669       opt_txn_db = nullptr;
1670     } else {
1671       delete db;
1672       db = nullptr;
1673     }
1674 #else
1675     delete db;
1676     db = nullptr;
1677 #endif  // ROCKSDB_LITE
1678   }
1679 
GetCfhROCKSDB_NAMESPACE::DBWithColumnFamilies1680   ColumnFamilyHandle* GetCfh(int64_t rand_num) {
1681     assert(num_hot > 0);
1682     size_t rand_offset = 0;
1683     if (!cfh_idx_to_prob.empty()) {
1684       assert(cfh_idx_to_prob.size() == num_hot);
1685       int sum = 0;
1686       while (sum + cfh_idx_to_prob[rand_offset] < rand_num % 100) {
1687         sum += cfh_idx_to_prob[rand_offset];
1688         ++rand_offset;
1689       }
1690       assert(rand_offset < cfh_idx_to_prob.size());
1691     } else {
1692       rand_offset = rand_num % num_hot;
1693     }
1694     return cfh[num_created.load(std::memory_order_acquire) - num_hot +
1695                rand_offset];
1696   }
1697 
1698   // stage: assume CF from 0 to stage * num_hot has be created. Need to create
1699   //        stage * num_hot + 1 to stage * (num_hot + 1).
CreateNewCfROCKSDB_NAMESPACE::DBWithColumnFamilies1700   void CreateNewCf(ColumnFamilyOptions options, int64_t stage) {
1701     MutexLock l(&create_cf_mutex);
1702     if ((stage + 1) * num_hot <= num_created) {
1703       // Already created.
1704       return;
1705     }
1706     auto new_num_created = num_created + num_hot;
1707     assert(new_num_created <= cfh.size());
1708     for (size_t i = num_created; i < new_num_created; i++) {
1709       Status s =
1710           db->CreateColumnFamily(options, ColumnFamilyName(i), &(cfh[i]));
1711       if (!s.ok()) {
1712         fprintf(stderr, "create column family error: %s\n",
1713                 s.ToString().c_str());
1714         abort();
1715       }
1716     }
1717     num_created.store(new_num_created, std::memory_order_release);
1718   }
1719 };
1720 
1721 // a class that reports stats to CSV file
1722 class ReporterAgent {
1723  public:
ReporterAgent(Env * env,const std::string & fname,uint64_t report_interval_secs)1724   ReporterAgent(Env* env, const std::string& fname,
1725                 uint64_t report_interval_secs)
1726       : env_(env),
1727         total_ops_done_(0),
1728         last_report_(0),
1729         report_interval_secs_(report_interval_secs),
1730         stop_(false) {
1731     auto s = env_->NewWritableFile(fname, &report_file_, EnvOptions());
1732     if (s.ok()) {
1733       s = report_file_->Append(Header() + "\n");
1734     }
1735     if (s.ok()) {
1736       s = report_file_->Flush();
1737     }
1738     if (!s.ok()) {
1739       fprintf(stderr, "Can't open %s: %s\n", fname.c_str(),
1740               s.ToString().c_str());
1741       abort();
1742     }
1743 
1744     reporting_thread_ = port::Thread([&]() { SleepAndReport(); });
1745   }
1746 
~ReporterAgent()1747   ~ReporterAgent() {
1748     {
1749       std::unique_lock<std::mutex> lk(mutex_);
1750       stop_ = true;
1751       stop_cv_.notify_all();
1752     }
1753     reporting_thread_.join();
1754   }
1755 
1756   // thread safe
ReportFinishedOps(int64_t num_ops)1757   void ReportFinishedOps(int64_t num_ops) {
1758     total_ops_done_.fetch_add(num_ops);
1759   }
1760 
1761  private:
Header() const1762   std::string Header() const { return "secs_elapsed,interval_qps"; }
SleepAndReport()1763   void SleepAndReport() {
1764     auto time_started = env_->NowMicros();
1765     while (true) {
1766       {
1767         std::unique_lock<std::mutex> lk(mutex_);
1768         if (stop_ ||
1769             stop_cv_.wait_for(lk, std::chrono::seconds(report_interval_secs_),
1770                               [&]() { return stop_; })) {
1771           // stopping
1772           break;
1773         }
1774         // else -> timeout, which means time for a report!
1775       }
1776       auto total_ops_done_snapshot = total_ops_done_.load();
1777       // round the seconds elapsed
1778       auto secs_elapsed =
1779           (env_->NowMicros() - time_started + kMicrosInSecond / 2) /
1780           kMicrosInSecond;
1781       std::string report = ToString(secs_elapsed) + "," +
1782                            ToString(total_ops_done_snapshot - last_report_) +
1783                            "\n";
1784       auto s = report_file_->Append(report);
1785       if (s.ok()) {
1786         s = report_file_->Flush();
1787       }
1788       if (!s.ok()) {
1789         fprintf(stderr,
1790                 "Can't write to report file (%s), stopping the reporting\n",
1791                 s.ToString().c_str());
1792         break;
1793       }
1794       last_report_ = total_ops_done_snapshot;
1795     }
1796   }
1797 
1798   Env* env_;
1799   std::unique_ptr<WritableFile> report_file_;
1800   std::atomic<int64_t> total_ops_done_;
1801   int64_t last_report_;
1802   const uint64_t report_interval_secs_;
1803   ROCKSDB_NAMESPACE::port::Thread reporting_thread_;
1804   std::mutex mutex_;
1805   // will notify on stop
1806   std::condition_variable stop_cv_;
1807   bool stop_;
1808 };
1809 
1810 enum OperationType : unsigned char {
1811   kRead = 0,
1812   kWrite,
1813   kDelete,
1814   kSeek,
1815   kMerge,
1816   kUpdate,
1817   kCompress,
1818   kUncompress,
1819   kCrc,
1820   kHash,
1821   kOthers
1822 };
1823 
1824 static std::unordered_map<OperationType, std::string, std::hash<unsigned char>>
1825                           OperationTypeString = {
1826   {kRead, "read"},
1827   {kWrite, "write"},
1828   {kDelete, "delete"},
1829   {kSeek, "seek"},
1830   {kMerge, "merge"},
1831   {kUpdate, "update"},
1832   {kCompress, "compress"},
1833   {kCompress, "uncompress"},
1834   {kCrc, "crc"},
1835   {kHash, "hash"},
1836   {kOthers, "op"}
1837 };
1838 
1839 class CombinedStats;
1840 class Stats {
1841  private:
1842   int id_;
1843   uint64_t start_;
1844   uint64_t sine_interval_;
1845   uint64_t finish_;
1846   double seconds_;
1847   uint64_t done_;
1848   uint64_t last_report_done_;
1849   uint64_t next_report_;
1850   uint64_t bytes_;
1851   uint64_t last_op_finish_;
1852   uint64_t last_report_finish_;
1853   std::unordered_map<OperationType, std::shared_ptr<HistogramImpl>,
1854                      std::hash<unsigned char>> hist_;
1855   std::string message_;
1856   bool exclude_from_merge_;
1857   ReporterAgent* reporter_agent_;  // does not own
1858   friend class CombinedStats;
1859 
1860  public:
Stats()1861   Stats() { Start(-1); }
1862 
SetReporterAgent(ReporterAgent * reporter_agent)1863   void SetReporterAgent(ReporterAgent* reporter_agent) {
1864     reporter_agent_ = reporter_agent;
1865   }
1866 
Start(int id)1867   void Start(int id) {
1868     id_ = id;
1869     next_report_ = FLAGS_stats_interval ? FLAGS_stats_interval : 100;
1870     last_op_finish_ = start_;
1871     hist_.clear();
1872     done_ = 0;
1873     last_report_done_ = 0;
1874     bytes_ = 0;
1875     seconds_ = 0;
1876     start_ = FLAGS_env->NowMicros();
1877     sine_interval_ = FLAGS_env->NowMicros();
1878     finish_ = start_;
1879     last_report_finish_ = start_;
1880     message_.clear();
1881     // When set, stats from this thread won't be merged with others.
1882     exclude_from_merge_ = false;
1883   }
1884 
Merge(const Stats & other)1885   void Merge(const Stats& other) {
1886     if (other.exclude_from_merge_)
1887       return;
1888 
1889     for (auto it = other.hist_.begin(); it != other.hist_.end(); ++it) {
1890       auto this_it = hist_.find(it->first);
1891       if (this_it != hist_.end()) {
1892         this_it->second->Merge(*(other.hist_.at(it->first)));
1893       } else {
1894         hist_.insert({ it->first, it->second });
1895       }
1896     }
1897 
1898     done_ += other.done_;
1899     bytes_ += other.bytes_;
1900     seconds_ += other.seconds_;
1901     if (other.start_ < start_) start_ = other.start_;
1902     if (other.finish_ > finish_) finish_ = other.finish_;
1903 
1904     // Just keep the messages from one thread
1905     if (message_.empty()) message_ = other.message_;
1906   }
1907 
Stop()1908   void Stop() {
1909     finish_ = FLAGS_env->NowMicros();
1910     seconds_ = (finish_ - start_) * 1e-6;
1911   }
1912 
AddMessage(Slice msg)1913   void AddMessage(Slice msg) {
1914     AppendWithSpace(&message_, msg);
1915   }
1916 
SetId(int id)1917   void SetId(int id) { id_ = id; }
SetExcludeFromMerge()1918   void SetExcludeFromMerge() { exclude_from_merge_ = true; }
1919 
PrintThreadStatus()1920   void PrintThreadStatus() {
1921     std::vector<ThreadStatus> thread_list;
1922     FLAGS_env->GetThreadList(&thread_list);
1923 
1924     fprintf(stderr, "\n%18s %10s %12s %20s %13s %45s %12s %s\n",
1925         "ThreadID", "ThreadType", "cfName", "Operation",
1926         "ElapsedTime", "Stage", "State", "OperationProperties");
1927 
1928     int64_t current_time = 0;
1929     FLAGS_env->GetCurrentTime(&current_time);
1930     for (auto ts : thread_list) {
1931       fprintf(stderr, "%18" PRIu64 " %10s %12s %20s %13s %45s %12s",
1932           ts.thread_id,
1933           ThreadStatus::GetThreadTypeName(ts.thread_type).c_str(),
1934           ts.cf_name.c_str(),
1935           ThreadStatus::GetOperationName(ts.operation_type).c_str(),
1936           ThreadStatus::MicrosToString(ts.op_elapsed_micros).c_str(),
1937           ThreadStatus::GetOperationStageName(ts.operation_stage).c_str(),
1938           ThreadStatus::GetStateName(ts.state_type).c_str());
1939 
1940       auto op_properties = ThreadStatus::InterpretOperationProperties(
1941           ts.operation_type, ts.op_properties);
1942       for (const auto& op_prop : op_properties) {
1943         fprintf(stderr, " %s %" PRIu64" |",
1944             op_prop.first.c_str(), op_prop.second);
1945       }
1946       fprintf(stderr, "\n");
1947     }
1948   }
1949 
ResetSineInterval()1950   void ResetSineInterval() {
1951     sine_interval_ = FLAGS_env->NowMicros();
1952   }
1953 
GetSineInterval()1954   uint64_t GetSineInterval() {
1955     return sine_interval_;
1956   }
1957 
GetStart()1958   uint64_t GetStart() {
1959     return start_;
1960   }
1961 
ResetLastOpTime()1962   void ResetLastOpTime() {
1963     // Set to now to avoid latency from calls to SleepForMicroseconds
1964     last_op_finish_ = FLAGS_env->NowMicros();
1965   }
1966 
FinishedOps(DBWithColumnFamilies * db_with_cfh,DB * db,int64_t num_ops,enum OperationType op_type=kOthers)1967   void FinishedOps(DBWithColumnFamilies* db_with_cfh, DB* db, int64_t num_ops,
1968                    enum OperationType op_type = kOthers) {
1969     if (reporter_agent_) {
1970       reporter_agent_->ReportFinishedOps(num_ops);
1971     }
1972     if (FLAGS_histogram) {
1973       uint64_t now = FLAGS_env->NowMicros();
1974       uint64_t micros = now - last_op_finish_;
1975 
1976       if (hist_.find(op_type) == hist_.end())
1977       {
1978         auto hist_temp = std::make_shared<HistogramImpl>();
1979         hist_.insert({op_type, std::move(hist_temp)});
1980       }
1981       hist_[op_type]->Add(micros);
1982 
1983       if (micros > 20000 && !FLAGS_stats_interval) {
1984         fprintf(stderr, "long op: %" PRIu64 " micros%30s\r", micros, "");
1985         fflush(stderr);
1986       }
1987       last_op_finish_ = now;
1988     }
1989 
1990     done_ += num_ops;
1991     if (done_ >= next_report_) {
1992       if (!FLAGS_stats_interval) {
1993         if      (next_report_ < 1000)   next_report_ += 100;
1994         else if (next_report_ < 5000)   next_report_ += 500;
1995         else if (next_report_ < 10000)  next_report_ += 1000;
1996         else if (next_report_ < 50000)  next_report_ += 5000;
1997         else if (next_report_ < 100000) next_report_ += 10000;
1998         else if (next_report_ < 500000) next_report_ += 50000;
1999         else                            next_report_ += 100000;
2000         fprintf(stderr, "... finished %" PRIu64 " ops%30s\r", done_, "");
2001       } else {
2002         uint64_t now = FLAGS_env->NowMicros();
2003         int64_t usecs_since_last = now - last_report_finish_;
2004 
2005         // Determine whether to print status where interval is either
2006         // each N operations or each N seconds.
2007 
2008         if (FLAGS_stats_interval_seconds &&
2009             usecs_since_last < (FLAGS_stats_interval_seconds * 1000000)) {
2010           // Don't check again for this many operations
2011           next_report_ += FLAGS_stats_interval;
2012 
2013         } else {
2014 
2015           fprintf(stderr,
2016                   "%s ... thread %d: (%" PRIu64 ",%" PRIu64 ") ops and "
2017                   "(%.1f,%.1f) ops/second in (%.6f,%.6f) seconds\n",
2018                   FLAGS_env->TimeToString(now/1000000).c_str(),
2019                   id_,
2020                   done_ - last_report_done_, done_,
2021                   (done_ - last_report_done_) /
2022                   (usecs_since_last / 1000000.0),
2023                   done_ / ((now - start_) / 1000000.0),
2024                   (now - last_report_finish_) / 1000000.0,
2025                   (now - start_) / 1000000.0);
2026 
2027           if (id_ == 0 && FLAGS_stats_per_interval) {
2028             std::string stats;
2029 
2030             if (db_with_cfh && db_with_cfh->num_created.load()) {
2031               for (size_t i = 0; i < db_with_cfh->num_created.load(); ++i) {
2032                 if (db->GetProperty(db_with_cfh->cfh[i], "rocksdb.cfstats",
2033                                     &stats))
2034                   fprintf(stderr, "%s\n", stats.c_str());
2035                 if (FLAGS_show_table_properties) {
2036                   for (int level = 0; level < FLAGS_num_levels; ++level) {
2037                     if (db->GetProperty(
2038                             db_with_cfh->cfh[i],
2039                             "rocksdb.aggregated-table-properties-at-level" +
2040                                 ToString(level),
2041                             &stats)) {
2042                       if (stats.find("# entries=0") == std::string::npos) {
2043                         fprintf(stderr, "Level[%d]: %s\n", level,
2044                                 stats.c_str());
2045                       }
2046                     }
2047                   }
2048                 }
2049               }
2050             } else if (db) {
2051               if (db->GetProperty("rocksdb.stats", &stats)) {
2052                 fprintf(stderr, "%s\n", stats.c_str());
2053               }
2054               if (FLAGS_show_table_properties) {
2055                 for (int level = 0; level < FLAGS_num_levels; ++level) {
2056                   if (db->GetProperty(
2057                           "rocksdb.aggregated-table-properties-at-level" +
2058                               ToString(level),
2059                           &stats)) {
2060                     if (stats.find("# entries=0") == std::string::npos) {
2061                       fprintf(stderr, "Level[%d]: %s\n", level, stats.c_str());
2062                     }
2063                   }
2064                 }
2065               }
2066             }
2067           }
2068 
2069           next_report_ += FLAGS_stats_interval;
2070           last_report_finish_ = now;
2071           last_report_done_ = done_;
2072         }
2073       }
2074       if (id_ == 0 && FLAGS_thread_status_per_interval) {
2075         PrintThreadStatus();
2076       }
2077       fflush(stderr);
2078     }
2079   }
2080 
AddBytes(int64_t n)2081   void AddBytes(int64_t n) {
2082     bytes_ += n;
2083   }
2084 
Report(const Slice & name)2085   void Report(const Slice& name) {
2086     // Pretend at least one op was done in case we are running a benchmark
2087     // that does not call FinishedOps().
2088     if (done_ < 1) done_ = 1;
2089 
2090     std::string extra;
2091     if (bytes_ > 0) {
2092       // Rate is computed on actual elapsed time, not the sum of per-thread
2093       // elapsed times.
2094       double elapsed = (finish_ - start_) * 1e-6;
2095       char rate[100];
2096       snprintf(rate, sizeof(rate), "%6.1f MB/s",
2097                (bytes_ / 1048576.0) / elapsed);
2098       extra = rate;
2099     }
2100     AppendWithSpace(&extra, message_);
2101     double elapsed = (finish_ - start_) * 1e-6;
2102     double throughput = (double)done_/elapsed;
2103 
2104     fprintf(stdout, "%-12s : %11.3f micros/op %ld ops/sec;%s%s\n",
2105             name.ToString().c_str(),
2106             seconds_ * 1e6 / done_,
2107             (long)throughput,
2108             (extra.empty() ? "" : " "),
2109             extra.c_str());
2110     if (FLAGS_histogram) {
2111       for (auto it = hist_.begin(); it != hist_.end(); ++it) {
2112         fprintf(stdout, "Microseconds per %s:\n%s\n",
2113                 OperationTypeString[it->first].c_str(),
2114                 it->second->ToString().c_str());
2115       }
2116     }
2117     if (FLAGS_report_file_operations) {
2118       ReportFileOpEnv* env = static_cast<ReportFileOpEnv*>(FLAGS_env);
2119       ReportFileOpCounters* counters = env->counters();
2120       fprintf(stdout, "Num files opened: %d\n",
2121               counters->open_counter_.load(std::memory_order_relaxed));
2122       fprintf(stdout, "Num Read(): %d\n",
2123               counters->read_counter_.load(std::memory_order_relaxed));
2124       fprintf(stdout, "Num Append(): %d\n",
2125               counters->append_counter_.load(std::memory_order_relaxed));
2126       fprintf(stdout, "Num bytes read: %" PRIu64 "\n",
2127               counters->bytes_read_.load(std::memory_order_relaxed));
2128       fprintf(stdout, "Num bytes written: %" PRIu64 "\n",
2129               counters->bytes_written_.load(std::memory_order_relaxed));
2130       env->reset();
2131     }
2132     fflush(stdout);
2133   }
2134 };
2135 
2136 class CombinedStats {
2137  public:
AddStats(const Stats & stat)2138   void AddStats(const Stats& stat) {
2139     uint64_t total_ops = stat.done_;
2140     uint64_t total_bytes_ = stat.bytes_;
2141     double elapsed;
2142 
2143     if (total_ops < 1) {
2144       total_ops = 1;
2145     }
2146 
2147     elapsed = (stat.finish_ - stat.start_) * 1e-6;
2148     throughput_ops_.emplace_back(total_ops / elapsed);
2149 
2150     if (total_bytes_ > 0) {
2151       double mbs = (total_bytes_ / 1048576.0);
2152       throughput_mbs_.emplace_back(mbs / elapsed);
2153     }
2154   }
2155 
Report(const std::string & bench_name)2156   void Report(const std::string& bench_name) {
2157     const char* name = bench_name.c_str();
2158     int num_runs = static_cast<int>(throughput_ops_.size());
2159 
2160     if (throughput_mbs_.size() == throughput_ops_.size()) {
2161       fprintf(stdout,
2162               "%s [AVG    %d runs] : %d ops/sec; %6.1f MB/sec\n"
2163               "%s [MEDIAN %d runs] : %d ops/sec; %6.1f MB/sec\n",
2164               name, num_runs, static_cast<int>(CalcAvg(throughput_ops_)),
2165               CalcAvg(throughput_mbs_), name, num_runs,
2166               static_cast<int>(CalcMedian(throughput_ops_)),
2167               CalcMedian(throughput_mbs_));
2168     } else {
2169       fprintf(stdout,
2170               "%s [AVG    %d runs] : %d ops/sec\n"
2171               "%s [MEDIAN %d runs] : %d ops/sec\n",
2172               name, num_runs, static_cast<int>(CalcAvg(throughput_ops_)), name,
2173               num_runs, static_cast<int>(CalcMedian(throughput_ops_)));
2174     }
2175   }
2176 
2177  private:
CalcAvg(std::vector<double> data)2178   double CalcAvg(std::vector<double> data) {
2179     double avg = 0;
2180     for (double x : data) {
2181       avg += x;
2182     }
2183     avg = avg / data.size();
2184     return avg;
2185   }
2186 
CalcMedian(std::vector<double> data)2187   double CalcMedian(std::vector<double> data) {
2188     assert(data.size() > 0);
2189     std::sort(data.begin(), data.end());
2190 
2191     size_t mid = data.size() / 2;
2192     if (data.size() % 2 == 1) {
2193       // Odd number of entries
2194       return data[mid];
2195     } else {
2196       // Even number of entries
2197       return (data[mid] + data[mid - 1]) / 2;
2198     }
2199   }
2200 
2201   std::vector<double> throughput_ops_;
2202   std::vector<double> throughput_mbs_;
2203 };
2204 
2205 class TimestampEmulator {
2206  private:
2207   std::atomic<uint64_t> timestamp_;
2208 
2209  public:
TimestampEmulator()2210   TimestampEmulator() : timestamp_(0) {}
Get() const2211   uint64_t Get() const { return timestamp_.load(); }
Inc()2212   void Inc() { timestamp_++; }
2213 };
2214 
2215 // State shared by all concurrent executions of the same benchmark.
2216 struct SharedState {
2217   port::Mutex mu;
2218   port::CondVar cv;
2219   int total;
2220   int perf_level;
2221   std::shared_ptr<RateLimiter> write_rate_limiter;
2222   std::shared_ptr<RateLimiter> read_rate_limiter;
2223 
2224   // Each thread goes through the following states:
2225   //    (1) initializing
2226   //    (2) waiting for others to be initialized
2227   //    (3) running
2228   //    (4) done
2229 
2230   long num_initialized;
2231   long num_done;
2232   bool start;
2233 
SharedStateROCKSDB_NAMESPACE::SharedState2234   SharedState() : cv(&mu), perf_level(FLAGS_perf_level) { }
2235 };
2236 
2237 // Per-thread state for concurrent executions of the same benchmark.
2238 struct ThreadState {
2239   int tid;             // 0..n-1 when running in n threads
2240   Random64 rand;         // Has different seeds for different threads
2241   Stats stats;
2242   SharedState* shared;
2243 
ThreadStateROCKSDB_NAMESPACE::ThreadState2244   /* implicit */ ThreadState(int index)
2245       : tid(index),
2246         rand((FLAGS_seed ? FLAGS_seed : 1000) + index) {
2247   }
2248 };
2249 
2250 class Duration {
2251  public:
Duration(uint64_t max_seconds,int64_t max_ops,int64_t ops_per_stage=0)2252   Duration(uint64_t max_seconds, int64_t max_ops, int64_t ops_per_stage = 0) {
2253     max_seconds_ = max_seconds;
2254     max_ops_= max_ops;
2255     ops_per_stage_ = (ops_per_stage > 0) ? ops_per_stage : max_ops;
2256     ops_ = 0;
2257     start_at_ = FLAGS_env->NowMicros();
2258   }
2259 
GetStage()2260   int64_t GetStage() { return std::min(ops_, max_ops_ - 1) / ops_per_stage_; }
2261 
Done(int64_t increment)2262   bool Done(int64_t increment) {
2263     if (increment <= 0) increment = 1;    // avoid Done(0) and infinite loops
2264     ops_ += increment;
2265 
2266     if (max_seconds_) {
2267       // Recheck every appx 1000 ops (exact iff increment is factor of 1000)
2268       auto granularity = FLAGS_ops_between_duration_checks;
2269       if ((ops_ / granularity) != ((ops_ - increment) / granularity)) {
2270         uint64_t now = FLAGS_env->NowMicros();
2271         return ((now - start_at_) / 1000000) >= max_seconds_;
2272       } else {
2273         return false;
2274       }
2275     } else {
2276       return ops_ > max_ops_;
2277     }
2278   }
2279 
2280  private:
2281   uint64_t max_seconds_;
2282   int64_t max_ops_;
2283   int64_t ops_per_stage_;
2284   int64_t ops_;
2285   uint64_t start_at_;
2286 };
2287 
2288 class Benchmark {
2289  private:
2290   std::shared_ptr<Cache> cache_;
2291   std::shared_ptr<Cache> compressed_cache_;
2292   std::shared_ptr<const FilterPolicy> filter_policy_;
2293   const SliceTransform* prefix_extractor_;
2294   DBWithColumnFamilies db_;
2295   std::vector<DBWithColumnFamilies> multi_dbs_;
2296   int64_t num_;
2297   int key_size_;
2298   int prefix_size_;
2299   int64_t keys_per_prefix_;
2300   int64_t entries_per_batch_;
2301   int64_t writes_before_delete_range_;
2302   int64_t writes_per_range_tombstone_;
2303   int64_t range_tombstone_width_;
2304   int64_t max_num_range_tombstones_;
2305   WriteOptions write_options_;
2306   Options open_options_;  // keep options around to properly destroy db later
2307 #ifndef ROCKSDB_LITE
2308   TraceOptions trace_options_;
2309   TraceOptions block_cache_trace_options_;
2310 #endif
2311   int64_t reads_;
2312   int64_t deletes_;
2313   double read_random_exp_range_;
2314   int64_t writes_;
2315   int64_t readwrites_;
2316   int64_t merge_keys_;
2317   bool report_file_operations_;
2318   bool use_blob_db_;
2319   std::vector<std::string> keys_;
2320 
2321   class ErrorHandlerListener : public EventListener {
2322    public:
2323 #ifndef ROCKSDB_LITE
ErrorHandlerListener()2324     ErrorHandlerListener()
2325         : mutex_(),
2326           cv_(&mutex_),
2327           no_auto_recovery_(false),
2328           recovery_complete_(false) {}
2329 
~ErrorHandlerListener()2330     ~ErrorHandlerListener() override {}
2331 
OnErrorRecoveryBegin(BackgroundErrorReason,Status,bool * auto_recovery)2332     void OnErrorRecoveryBegin(BackgroundErrorReason /*reason*/,
2333                               Status /*bg_error*/,
2334                               bool* auto_recovery) override {
2335       if (*auto_recovery && no_auto_recovery_) {
2336         *auto_recovery = false;
2337       }
2338     }
2339 
OnErrorRecoveryCompleted(Status)2340     void OnErrorRecoveryCompleted(Status /*old_bg_error*/) override {
2341       InstrumentedMutexLock l(&mutex_);
2342       recovery_complete_ = true;
2343       cv_.SignalAll();
2344     }
2345 
WaitForRecovery(uint64_t abs_time_us)2346     bool WaitForRecovery(uint64_t abs_time_us) {
2347       InstrumentedMutexLock l(&mutex_);
2348       if (!recovery_complete_) {
2349         cv_.TimedWait(abs_time_us);
2350       }
2351       if (recovery_complete_) {
2352         recovery_complete_ = false;
2353         return true;
2354       }
2355       return false;
2356     }
2357 
EnableAutoRecovery(bool enable=true)2358     void EnableAutoRecovery(bool enable = true) { no_auto_recovery_ = !enable; }
2359 
2360    private:
2361     InstrumentedMutex mutex_;
2362     InstrumentedCondVar cv_;
2363     bool no_auto_recovery_;
2364     bool recovery_complete_;
2365 #else   // ROCKSDB_LITE
2366     bool WaitForRecovery(uint64_t /*abs_time_us*/) { return true; }
2367     void EnableAutoRecovery(bool /*enable*/) {}
2368 #endif  // ROCKSDB_LITE
2369   };
2370 
2371   std::shared_ptr<ErrorHandlerListener> listener_;
2372 
SanityCheck()2373   bool SanityCheck() {
2374     if (FLAGS_compression_ratio > 1) {
2375       fprintf(stderr, "compression_ratio should be between 0 and 1\n");
2376       return false;
2377     }
2378     return true;
2379   }
2380 
CompressSlice(const CompressionInfo & compression_info,const Slice & input,std::string * compressed)2381   inline bool CompressSlice(const CompressionInfo& compression_info,
2382                             const Slice& input, std::string* compressed) {
2383     bool ok = true;
2384     switch (FLAGS_compression_type_e) {
2385       case ROCKSDB_NAMESPACE::kSnappyCompression:
2386         ok = Snappy_Compress(compression_info, input.data(), input.size(),
2387                              compressed);
2388         break;
2389       case ROCKSDB_NAMESPACE::kZlibCompression:
2390         ok = Zlib_Compress(compression_info, 2, input.data(), input.size(),
2391                            compressed);
2392         break;
2393       case ROCKSDB_NAMESPACE::kBZip2Compression:
2394         ok = BZip2_Compress(compression_info, 2, input.data(), input.size(),
2395                             compressed);
2396         break;
2397       case ROCKSDB_NAMESPACE::kLZ4Compression:
2398         ok = LZ4_Compress(compression_info, 2, input.data(), input.size(),
2399                           compressed);
2400         break;
2401       case ROCKSDB_NAMESPACE::kLZ4HCCompression:
2402         ok = LZ4HC_Compress(compression_info, 2, input.data(), input.size(),
2403                             compressed);
2404         break;
2405       case ROCKSDB_NAMESPACE::kXpressCompression:
2406         ok = XPRESS_Compress(input.data(),
2407           input.size(), compressed);
2408         break;
2409       case ROCKSDB_NAMESPACE::kZSTD:
2410         ok = ZSTD_Compress(compression_info, input.data(), input.size(),
2411                            compressed);
2412         break;
2413       default:
2414         ok = false;
2415     }
2416     return ok;
2417   }
2418 
PrintHeader()2419   void PrintHeader() {
2420     PrintEnvironment();
2421     fprintf(stdout, "Keys:       %d bytes each\n", FLAGS_key_size);
2422     auto avg_value_size = FLAGS_value_size;
2423     if (FLAGS_value_size_distribution_type_e == kFixed) {
2424       fprintf(stdout, "Values:     %d bytes each (%d bytes after compression)\n",
2425               avg_value_size,
2426               static_cast<int>(avg_value_size * FLAGS_compression_ratio + 0.5));
2427     } else {
2428       avg_value_size = (FLAGS_value_size_min + FLAGS_value_size_max) / 2;
2429       fprintf(stdout, "Values:     %d avg bytes each (%d bytes after compression)\n",
2430               avg_value_size,
2431               static_cast<int>(avg_value_size * FLAGS_compression_ratio + 0.5));
2432       fprintf(stdout, "Values Distribution: %s (min: %d, max: %d)\n",
2433               FLAGS_value_size_distribution_type.c_str(),
2434               FLAGS_value_size_min, FLAGS_value_size_max);
2435     }
2436     fprintf(stdout, "Entries:    %" PRIu64 "\n", num_);
2437     fprintf(stdout, "Prefix:    %d bytes\n", FLAGS_prefix_size);
2438     fprintf(stdout, "Keys per prefix:    %" PRIu64 "\n", keys_per_prefix_);
2439     fprintf(stdout, "RawSize:    %.1f MB (estimated)\n",
2440             ((static_cast<int64_t>(FLAGS_key_size + avg_value_size) * num_)
2441              / 1048576.0));
2442     fprintf(stdout, "FileSize:   %.1f MB (estimated)\n",
2443             (((FLAGS_key_size + avg_value_size * FLAGS_compression_ratio)
2444               * num_)
2445              / 1048576.0));
2446     fprintf(stdout, "Write rate: %" PRIu64 " bytes/second\n",
2447             FLAGS_benchmark_write_rate_limit);
2448     fprintf(stdout, "Read rate: %" PRIu64 " ops/second\n",
2449             FLAGS_benchmark_read_rate_limit);
2450     if (FLAGS_enable_numa) {
2451       fprintf(stderr, "Running in NUMA enabled mode.\n");
2452 #ifndef NUMA
2453       fprintf(stderr, "NUMA is not defined in the system.\n");
2454       exit(1);
2455 #else
2456       if (numa_available() == -1) {
2457         fprintf(stderr, "NUMA is not supported by the system.\n");
2458         exit(1);
2459       }
2460 #endif
2461     }
2462 
2463     auto compression = CompressionTypeToString(FLAGS_compression_type_e);
2464     fprintf(stdout, "Compression: %s\n", compression.c_str());
2465     fprintf(stdout, "Compression sampling rate: %" PRId64 "\n",
2466             FLAGS_sample_for_compression);
2467 
2468     switch (FLAGS_rep_factory) {
2469       case kPrefixHash:
2470         fprintf(stdout, "Memtablerep: prefix_hash\n");
2471         break;
2472       case kSkipList:
2473         fprintf(stdout, "Memtablerep: skip_list\n");
2474         break;
2475       case kVectorRep:
2476         fprintf(stdout, "Memtablerep: vector\n");
2477         break;
2478       case kHashLinkedList:
2479         fprintf(stdout, "Memtablerep: hash_linkedlist\n");
2480         break;
2481     }
2482     fprintf(stdout, "Perf Level: %d\n", FLAGS_perf_level);
2483 
2484     PrintWarnings(compression.c_str());
2485     fprintf(stdout, "------------------------------------------------\n");
2486   }
2487 
PrintWarnings(const char * compression)2488   void PrintWarnings(const char* compression) {
2489 #if defined(__GNUC__) && !defined(__OPTIMIZE__)
2490     fprintf(stdout,
2491             "WARNING: Optimization is disabled: benchmarks unnecessarily slow\n"
2492             );
2493 #endif
2494 #ifndef NDEBUG
2495     fprintf(stdout,
2496             "WARNING: Assertions are enabled; benchmarks unnecessarily slow\n");
2497 #endif
2498     if (FLAGS_compression_type_e != ROCKSDB_NAMESPACE::kNoCompression) {
2499       // The test string should not be too small.
2500       const int len = FLAGS_block_size;
2501       std::string input_str(len, 'y');
2502       std::string compressed;
2503       CompressionOptions opts;
2504       CompressionContext context(FLAGS_compression_type_e);
2505       CompressionInfo info(opts, context, CompressionDict::GetEmptyDict(),
2506                            FLAGS_compression_type_e,
2507                            FLAGS_sample_for_compression);
2508       bool result = CompressSlice(info, Slice(input_str), &compressed);
2509 
2510       if (!result) {
2511         fprintf(stdout, "WARNING: %s compression is not enabled\n",
2512                 compression);
2513       } else if (compressed.size() >= input_str.size()) {
2514         fprintf(stdout, "WARNING: %s compression is not effective\n",
2515                 compression);
2516       }
2517     }
2518   }
2519 
2520 // Current the following isn't equivalent to OS_LINUX.
2521 #if defined(__linux)
TrimSpace(Slice s)2522   static Slice TrimSpace(Slice s) {
2523     unsigned int start = 0;
2524     while (start < s.size() && isspace(s[start])) {
2525       start++;
2526     }
2527     unsigned int limit = static_cast<unsigned int>(s.size());
2528     while (limit > start && isspace(s[limit-1])) {
2529       limit--;
2530     }
2531     return Slice(s.data() + start, limit - start);
2532   }
2533 #endif
2534 
PrintEnvironment()2535   void PrintEnvironment() {
2536     fprintf(stderr, "RocksDB:    version %d.%d\n",
2537             kMajorVersion, kMinorVersion);
2538 
2539 #if defined(__linux)
2540     time_t now = time(nullptr);
2541     char buf[52];
2542     // Lint complains about ctime() usage, so replace it with ctime_r(). The
2543     // requirement is to provide a buffer which is at least 26 bytes.
2544     fprintf(stderr, "Date:       %s",
2545             ctime_r(&now, buf));  // ctime_r() adds newline
2546 
2547     FILE* cpuinfo = fopen("/proc/cpuinfo", "r");
2548     if (cpuinfo != nullptr) {
2549       char line[1000];
2550       int num_cpus = 0;
2551       std::string cpu_type;
2552       std::string cache_size;
2553       while (fgets(line, sizeof(line), cpuinfo) != nullptr) {
2554         const char* sep = strchr(line, ':');
2555         if (sep == nullptr) {
2556           continue;
2557         }
2558         Slice key = TrimSpace(Slice(line, sep - 1 - line));
2559         Slice val = TrimSpace(Slice(sep + 1));
2560         if (key == "model name") {
2561           ++num_cpus;
2562           cpu_type = val.ToString();
2563         } else if (key == "cache size") {
2564           cache_size = val.ToString();
2565         }
2566       }
2567       fclose(cpuinfo);
2568       fprintf(stderr, "CPU:        %d * %s\n", num_cpus, cpu_type.c_str());
2569       fprintf(stderr, "CPUCache:   %s\n", cache_size.c_str());
2570     }
2571 #endif
2572   }
2573 
KeyExpired(const TimestampEmulator * timestamp_emulator,const Slice & key)2574   static bool KeyExpired(const TimestampEmulator* timestamp_emulator,
2575                          const Slice& key) {
2576     const char* pos = key.data();
2577     pos += 8;
2578     uint64_t timestamp = 0;
2579     if (port::kLittleEndian) {
2580       int bytes_to_fill = 8;
2581       for (int i = 0; i < bytes_to_fill; ++i) {
2582         timestamp |= (static_cast<uint64_t>(static_cast<unsigned char>(pos[i]))
2583                       << ((bytes_to_fill - i - 1) << 3));
2584       }
2585     } else {
2586       memcpy(&timestamp, pos, sizeof(timestamp));
2587     }
2588     return timestamp_emulator->Get() - timestamp > FLAGS_time_range;
2589   }
2590 
2591   class ExpiredTimeFilter : public CompactionFilter {
2592    public:
ExpiredTimeFilter(const std::shared_ptr<TimestampEmulator> & timestamp_emulator)2593     explicit ExpiredTimeFilter(
2594         const std::shared_ptr<TimestampEmulator>& timestamp_emulator)
2595         : timestamp_emulator_(timestamp_emulator) {}
Filter(int,const Slice & key,const Slice &,std::string *,bool *) const2596     bool Filter(int /*level*/, const Slice& key,
2597                 const Slice& /*existing_value*/, std::string* /*new_value*/,
2598                 bool* /*value_changed*/) const override {
2599       return KeyExpired(timestamp_emulator_.get(), key);
2600     }
Name() const2601     const char* Name() const override { return "ExpiredTimeFilter"; }
2602 
2603    private:
2604     std::shared_ptr<TimestampEmulator> timestamp_emulator_;
2605   };
2606 
2607   class KeepFilter : public CompactionFilter {
2608    public:
Filter(int,const Slice &,const Slice &,std::string *,bool *) const2609     bool Filter(int /*level*/, const Slice& /*key*/, const Slice& /*value*/,
2610                 std::string* /*new_value*/,
2611                 bool* /*value_changed*/) const override {
2612       return false;
2613     }
2614 
Name() const2615     const char* Name() const override { return "KeepFilter"; }
2616   };
2617 
NewCache(int64_t capacity)2618   std::shared_ptr<Cache> NewCache(int64_t capacity) {
2619     if (capacity <= 0) {
2620       return nullptr;
2621     }
2622     if (FLAGS_use_clock_cache) {
2623       auto cache = NewClockCache(static_cast<size_t>(capacity),
2624                                  FLAGS_cache_numshardbits);
2625       if (!cache) {
2626         fprintf(stderr, "Clock cache not supported.");
2627         exit(1);
2628       }
2629       return cache;
2630     } else {
2631       return NewLRUCache(
2632           static_cast<size_t>(capacity), FLAGS_cache_numshardbits,
2633           false /*strict_capacity_limit*/, FLAGS_cache_high_pri_pool_ratio);
2634     }
2635   }
2636 
2637  public:
Benchmark()2638   Benchmark()
2639       : cache_(NewCache(FLAGS_cache_size)),
2640         compressed_cache_(NewCache(FLAGS_compressed_cache_size)),
2641         filter_policy_(FLAGS_bloom_bits >= 0
2642                            ? NewBloomFilterPolicy(FLAGS_bloom_bits,
2643                                                   FLAGS_use_block_based_filter)
2644                            : nullptr),
2645         prefix_extractor_(NewFixedPrefixTransform(FLAGS_prefix_size)),
2646         num_(FLAGS_num),
2647         key_size_(FLAGS_key_size),
2648         prefix_size_(FLAGS_prefix_size),
2649         keys_per_prefix_(FLAGS_keys_per_prefix),
2650         entries_per_batch_(1),
2651         reads_(FLAGS_reads < 0 ? FLAGS_num : FLAGS_reads),
2652         read_random_exp_range_(0.0),
2653         writes_(FLAGS_writes < 0 ? FLAGS_num : FLAGS_writes),
2654         readwrites_(
2655             (FLAGS_writes < 0 && FLAGS_reads < 0)
2656                 ? FLAGS_num
2657                 : ((FLAGS_writes > FLAGS_reads) ? FLAGS_writes : FLAGS_reads)),
2658         merge_keys_(FLAGS_merge_keys < 0 ? FLAGS_num : FLAGS_merge_keys),
2659         report_file_operations_(FLAGS_report_file_operations),
2660 #ifndef ROCKSDB_LITE
2661         use_blob_db_(FLAGS_use_blob_db)
2662 #else
2663         use_blob_db_(false)
2664 #endif  // !ROCKSDB_LITE
2665   {
2666     // use simcache instead of cache
2667     if (FLAGS_simcache_size >= 0) {
2668       if (FLAGS_cache_numshardbits >= 1) {
2669         cache_ =
2670             NewSimCache(cache_, FLAGS_simcache_size, FLAGS_cache_numshardbits);
2671       } else {
2672         cache_ = NewSimCache(cache_, FLAGS_simcache_size, 0);
2673       }
2674     }
2675 
2676     if (report_file_operations_) {
2677       if (!FLAGS_hdfs.empty()) {
2678         fprintf(stderr,
2679                 "--hdfs and --report_file_operations cannot be enabled "
2680                 "at the same time");
2681         exit(1);
2682       }
2683       FLAGS_env = new ReportFileOpEnv(FLAGS_env);
2684     }
2685 
2686     if (FLAGS_prefix_size > FLAGS_key_size) {
2687       fprintf(stderr, "prefix size is larger than key size");
2688       exit(1);
2689     }
2690 
2691     std::vector<std::string> files;
2692     FLAGS_env->GetChildren(FLAGS_db, &files);
2693     for (size_t i = 0; i < files.size(); i++) {
2694       if (Slice(files[i]).starts_with("heap-")) {
2695         FLAGS_env->DeleteFile(FLAGS_db + "/" + files[i]);
2696       }
2697     }
2698     if (!FLAGS_use_existing_db) {
2699       Options options;
2700       options.env = FLAGS_env;
2701       if (!FLAGS_wal_dir.empty()) {
2702         options.wal_dir = FLAGS_wal_dir;
2703       }
2704 #ifndef ROCKSDB_LITE
2705       if (use_blob_db_) {
2706         blob_db::DestroyBlobDB(FLAGS_db, options, blob_db::BlobDBOptions());
2707       }
2708 #endif  // !ROCKSDB_LITE
2709       DestroyDB(FLAGS_db, options);
2710       if (!FLAGS_wal_dir.empty()) {
2711         FLAGS_env->DeleteDir(FLAGS_wal_dir);
2712       }
2713 
2714       if (FLAGS_num_multi_db > 1) {
2715         FLAGS_env->CreateDir(FLAGS_db);
2716         if (!FLAGS_wal_dir.empty()) {
2717           FLAGS_env->CreateDir(FLAGS_wal_dir);
2718         }
2719       }
2720     }
2721 
2722     listener_.reset(new ErrorHandlerListener());
2723   }
2724 
~Benchmark()2725   ~Benchmark() {
2726     db_.DeleteDBs();
2727     delete prefix_extractor_;
2728     if (cache_.get() != nullptr) {
2729       // this will leak, but we're shutting down so nobody cares
2730       cache_->DisownData();
2731     }
2732   }
2733 
AllocateKey(std::unique_ptr<const char[]> * key_guard)2734   Slice AllocateKey(std::unique_ptr<const char[]>* key_guard) {
2735     char* data = new char[key_size_];
2736     const char* const_data = data;
2737     key_guard->reset(const_data);
2738     return Slice(key_guard->get(), key_size_);
2739   }
2740 
2741   // Generate key according to the given specification and random number.
2742   // The resulting key will have the following format (if keys_per_prefix_
2743   // is positive), extra trailing bytes are either cut off or padded with '0'.
2744   // The prefix value is derived from key value.
2745   //   ----------------------------
2746   //   | prefix 00000 | key 00000 |
2747   //   ----------------------------
2748   // If keys_per_prefix_ is 0, the key is simply a binary representation of
2749   // random number followed by trailing '0's
2750   //   ----------------------------
2751   //   |        key 00000         |
2752   //   ----------------------------
GenerateKeyFromInt(uint64_t v,int64_t num_keys,Slice * key)2753   void GenerateKeyFromInt(uint64_t v, int64_t num_keys, Slice* key) {
2754     if (!keys_.empty()) {
2755       assert(FLAGS_use_existing_keys);
2756       assert(keys_.size() == static_cast<size_t>(num_keys));
2757       assert(v < static_cast<uint64_t>(num_keys));
2758       *key = keys_[v];
2759       return;
2760     }
2761     char* start = const_cast<char*>(key->data());
2762     char* pos = start;
2763     if (keys_per_prefix_ > 0) {
2764       int64_t num_prefix = num_keys / keys_per_prefix_;
2765       int64_t prefix = v % num_prefix;
2766       int bytes_to_fill = std::min(prefix_size_, 8);
2767       if (port::kLittleEndian) {
2768         for (int i = 0; i < bytes_to_fill; ++i) {
2769           pos[i] = (prefix >> ((bytes_to_fill - i - 1) << 3)) & 0xFF;
2770         }
2771       } else {
2772         memcpy(pos, static_cast<void*>(&prefix), bytes_to_fill);
2773       }
2774       if (prefix_size_ > 8) {
2775         // fill the rest with 0s
2776         memset(pos + 8, '0', prefix_size_ - 8);
2777       }
2778       pos += prefix_size_;
2779     }
2780 
2781     int bytes_to_fill = std::min(key_size_ - static_cast<int>(pos - start), 8);
2782     if (port::kLittleEndian) {
2783       for (int i = 0; i < bytes_to_fill; ++i) {
2784         pos[i] = (v >> ((bytes_to_fill - i - 1) << 3)) & 0xFF;
2785       }
2786     } else {
2787       memcpy(pos, static_cast<void*>(&v), bytes_to_fill);
2788     }
2789     pos += bytes_to_fill;
2790     if (key_size_ > pos - start) {
2791       memset(pos, '0', key_size_ - (pos - start));
2792     }
2793   }
2794 
GenerateKeyFromIntForSeek(uint64_t v,int64_t num_keys,Slice * key)2795   void GenerateKeyFromIntForSeek(uint64_t v, int64_t num_keys, Slice* key) {
2796     GenerateKeyFromInt(v, num_keys, key);
2797     if (FLAGS_seek_missing_prefix) {
2798       assert(prefix_size_ > 8);
2799       char* key_ptr = const_cast<char*>(key->data());
2800       // This rely on GenerateKeyFromInt filling paddings with '0's.
2801       // Putting a '1' will create a non-existing prefix.
2802       key_ptr[8] = '1';
2803     }
2804   }
2805 
GetPathForMultiple(std::string base_name,size_t id)2806   std::string GetPathForMultiple(std::string base_name, size_t id) {
2807     if (!base_name.empty()) {
2808 #ifndef OS_WIN
2809       if (base_name.back() != '/') {
2810         base_name += '/';
2811       }
2812 #else
2813       if (base_name.back() != '\\') {
2814         base_name += '\\';
2815       }
2816 #endif
2817     }
2818     return base_name + ToString(id);
2819   }
2820 
VerifyDBFromDB(std::string & truth_db_name)2821   void VerifyDBFromDB(std::string& truth_db_name) {
2822     DBWithColumnFamilies truth_db;
2823     auto s = DB::OpenForReadOnly(open_options_, truth_db_name, &truth_db.db);
2824     if (!s.ok()) {
2825       fprintf(stderr, "open error: %s\n", s.ToString().c_str());
2826       exit(1);
2827     }
2828     ReadOptions ro;
2829     ro.total_order_seek = true;
2830     std::unique_ptr<Iterator> truth_iter(truth_db.db->NewIterator(ro));
2831     std::unique_ptr<Iterator> db_iter(db_.db->NewIterator(ro));
2832     // Verify that all the key/values in truth_db are retrivable in db with
2833     // ::Get
2834     fprintf(stderr, "Verifying db >= truth_db with ::Get...\n");
2835     for (truth_iter->SeekToFirst(); truth_iter->Valid(); truth_iter->Next()) {
2836       std::string value;
2837       s = db_.db->Get(ro, truth_iter->key(), &value);
2838       assert(s.ok());
2839       // TODO(myabandeh): provide debugging hints
2840       assert(Slice(value) == truth_iter->value());
2841     }
2842     // Verify that the db iterator does not give any extra key/value
2843     fprintf(stderr, "Verifying db == truth_db...\n");
2844     for (db_iter->SeekToFirst(), truth_iter->SeekToFirst(); db_iter->Valid();
2845          db_iter->Next(), truth_iter->Next()) {
2846       assert(truth_iter->Valid());
2847       assert(truth_iter->value() == db_iter->value());
2848     }
2849     // No more key should be left unchecked in truth_db
2850     assert(!truth_iter->Valid());
2851     fprintf(stderr, "...Verified\n");
2852   }
2853 
Run()2854   void Run() {
2855     if (!SanityCheck()) {
2856       exit(1);
2857     }
2858     Open(&open_options_);
2859     PrintHeader();
2860     std::stringstream benchmark_stream(FLAGS_benchmarks);
2861     std::string name;
2862     std::unique_ptr<ExpiredTimeFilter> filter;
2863     while (std::getline(benchmark_stream, name, ',')) {
2864       // Sanitize parameters
2865       num_ = FLAGS_num;
2866       reads_ = (FLAGS_reads < 0 ? FLAGS_num : FLAGS_reads);
2867       writes_ = (FLAGS_writes < 0 ? FLAGS_num : FLAGS_writes);
2868       deletes_ = (FLAGS_deletes < 0 ? FLAGS_num : FLAGS_deletes);
2869       value_size = FLAGS_value_size;
2870       key_size_ = FLAGS_key_size;
2871       entries_per_batch_ = FLAGS_batch_size;
2872       writes_before_delete_range_ = FLAGS_writes_before_delete_range;
2873       writes_per_range_tombstone_ = FLAGS_writes_per_range_tombstone;
2874       range_tombstone_width_ = FLAGS_range_tombstone_width;
2875       max_num_range_tombstones_ = FLAGS_max_num_range_tombstones;
2876       write_options_ = WriteOptions();
2877       read_random_exp_range_ = FLAGS_read_random_exp_range;
2878       if (FLAGS_sync) {
2879         write_options_.sync = true;
2880       }
2881       write_options_.disableWAL = FLAGS_disable_wal;
2882 
2883       void (Benchmark::*method)(ThreadState*) = nullptr;
2884       void (Benchmark::*post_process_method)() = nullptr;
2885 
2886       bool fresh_db = false;
2887       int num_threads = FLAGS_threads;
2888 
2889       int num_repeat = 1;
2890       int num_warmup = 0;
2891       if (!name.empty() && *name.rbegin() == ']') {
2892         auto it = name.find('[');
2893         if (it == std::string::npos) {
2894           fprintf(stderr, "unknown benchmark arguments '%s'\n", name.c_str());
2895           exit(1);
2896         }
2897         std::string args = name.substr(it + 1);
2898         args.resize(args.size() - 1);
2899         name.resize(it);
2900 
2901         std::string bench_arg;
2902         std::stringstream args_stream(args);
2903         while (std::getline(args_stream, bench_arg, '-')) {
2904           if (bench_arg.empty()) {
2905             continue;
2906           }
2907           if (bench_arg[0] == 'X') {
2908             // Repeat the benchmark n times
2909             std::string num_str = bench_arg.substr(1);
2910             num_repeat = std::stoi(num_str);
2911           } else if (bench_arg[0] == 'W') {
2912             // Warm up the benchmark for n times
2913             std::string num_str = bench_arg.substr(1);
2914             num_warmup = std::stoi(num_str);
2915           }
2916         }
2917       }
2918 
2919       // Both fillseqdeterministic and filluniquerandomdeterministic
2920       // fill the levels except the max level with UNIQUE_RANDOM
2921       // and fill the max level with fillseq and filluniquerandom, respectively
2922       if (name == "fillseqdeterministic" ||
2923           name == "filluniquerandomdeterministic") {
2924         if (!FLAGS_disable_auto_compactions) {
2925           fprintf(stderr,
2926                   "Please disable_auto_compactions in FillDeterministic "
2927                   "benchmark\n");
2928           exit(1);
2929         }
2930         if (num_threads > 1) {
2931           fprintf(stderr,
2932                   "filldeterministic multithreaded not supported"
2933                   ", use 1 thread\n");
2934           num_threads = 1;
2935         }
2936         fresh_db = true;
2937         if (name == "fillseqdeterministic") {
2938           method = &Benchmark::WriteSeqDeterministic;
2939         } else {
2940           method = &Benchmark::WriteUniqueRandomDeterministic;
2941         }
2942       } else if (name == "fillseq") {
2943         fresh_db = true;
2944         method = &Benchmark::WriteSeq;
2945       } else if (name == "fillbatch") {
2946         fresh_db = true;
2947         entries_per_batch_ = 1000;
2948         method = &Benchmark::WriteSeq;
2949       } else if (name == "fillrandom") {
2950         fresh_db = true;
2951         method = &Benchmark::WriteRandom;
2952       } else if (name == "filluniquerandom") {
2953         fresh_db = true;
2954         if (num_threads > 1) {
2955           fprintf(stderr,
2956                   "filluniquerandom multithreaded not supported"
2957                   ", use 1 thread");
2958           num_threads = 1;
2959         }
2960         method = &Benchmark::WriteUniqueRandom;
2961       } else if (name == "overwrite") {
2962         method = &Benchmark::WriteRandom;
2963       } else if (name == "fillsync") {
2964         fresh_db = true;
2965         num_ /= 1000;
2966         write_options_.sync = true;
2967         method = &Benchmark::WriteRandom;
2968       } else if (name == "fill100K") {
2969         fresh_db = true;
2970         num_ /= 1000;
2971         value_size = 100 * 1000;
2972         method = &Benchmark::WriteRandom;
2973       } else if (name == "readseq") {
2974         method = &Benchmark::ReadSequential;
2975       } else if (name == "readtorowcache") {
2976         if (!FLAGS_use_existing_keys || !FLAGS_row_cache_size) {
2977           fprintf(stderr,
2978                   "Please set use_existing_keys to true and specify a "
2979                   "row cache size in readtorowcache benchmark\n");
2980           exit(1);
2981         }
2982         method = &Benchmark::ReadToRowCache;
2983       } else if (name == "readtocache") {
2984         method = &Benchmark::ReadSequential;
2985         num_threads = 1;
2986         reads_ = num_;
2987       } else if (name == "readreverse") {
2988         method = &Benchmark::ReadReverse;
2989       } else if (name == "readrandom") {
2990         if (FLAGS_multiread_stride) {
2991           fprintf(stderr, "entries_per_batch = %" PRIi64 "\n",
2992                   entries_per_batch_);
2993         }
2994         method = &Benchmark::ReadRandom;
2995       } else if (name == "readrandomfast") {
2996         method = &Benchmark::ReadRandomFast;
2997       } else if (name == "multireadrandom") {
2998         fprintf(stderr, "entries_per_batch = %" PRIi64 "\n",
2999                 entries_per_batch_);
3000         method = &Benchmark::MultiReadRandom;
3001       } else if (name == "mixgraph") {
3002         method = &Benchmark::MixGraph;
3003       } else if (name == "readmissing") {
3004         ++key_size_;
3005         method = &Benchmark::ReadRandom;
3006       } else if (name == "newiterator") {
3007         method = &Benchmark::IteratorCreation;
3008       } else if (name == "newiteratorwhilewriting") {
3009         num_threads++;  // Add extra thread for writing
3010         method = &Benchmark::IteratorCreationWhileWriting;
3011       } else if (name == "seekrandom") {
3012         method = &Benchmark::SeekRandom;
3013       } else if (name == "seekrandomwhilewriting") {
3014         num_threads++;  // Add extra thread for writing
3015         method = &Benchmark::SeekRandomWhileWriting;
3016       } else if (name == "seekrandomwhilemerging") {
3017         num_threads++;  // Add extra thread for merging
3018         method = &Benchmark::SeekRandomWhileMerging;
3019       } else if (name == "readrandomsmall") {
3020         reads_ /= 1000;
3021         method = &Benchmark::ReadRandom;
3022       } else if (name == "deleteseq") {
3023         method = &Benchmark::DeleteSeq;
3024       } else if (name == "deleterandom") {
3025         method = &Benchmark::DeleteRandom;
3026       } else if (name == "readwhilewriting") {
3027         num_threads++;  // Add extra thread for writing
3028         method = &Benchmark::ReadWhileWriting;
3029       } else if (name == "readwhilemerging") {
3030         num_threads++;  // Add extra thread for writing
3031         method = &Benchmark::ReadWhileMerging;
3032       } else if (name == "readwhilescanning") {
3033         num_threads++;  // Add extra thread for scaning
3034         method = &Benchmark::ReadWhileScanning;
3035       } else if (name == "readrandomwriterandom") {
3036         method = &Benchmark::ReadRandomWriteRandom;
3037       } else if (name == "readrandommergerandom") {
3038         if (FLAGS_merge_operator.empty()) {
3039           fprintf(stdout, "%-12s : skipped (--merge_operator is unknown)\n",
3040                   name.c_str());
3041           exit(1);
3042         }
3043         method = &Benchmark::ReadRandomMergeRandom;
3044       } else if (name == "updaterandom") {
3045         method = &Benchmark::UpdateRandom;
3046       } else if (name == "xorupdaterandom") {
3047         method = &Benchmark::XORUpdateRandom;
3048       } else if (name == "appendrandom") {
3049         method = &Benchmark::AppendRandom;
3050       } else if (name == "mergerandom") {
3051         if (FLAGS_merge_operator.empty()) {
3052           fprintf(stdout, "%-12s : skipped (--merge_operator is unknown)\n",
3053                   name.c_str());
3054           exit(1);
3055         }
3056         method = &Benchmark::MergeRandom;
3057       } else if (name == "randomwithverify") {
3058         method = &Benchmark::RandomWithVerify;
3059       } else if (name == "fillseekseq") {
3060         method = &Benchmark::WriteSeqSeekSeq;
3061       } else if (name == "compact") {
3062         method = &Benchmark::Compact;
3063       } else if (name == "compactall") {
3064         CompactAll();
3065       } else if (name == "crc32c") {
3066         method = &Benchmark::Crc32c;
3067       } else if (name == "xxhash") {
3068         method = &Benchmark::xxHash;
3069       } else if (name == "acquireload") {
3070         method = &Benchmark::AcquireLoad;
3071       } else if (name == "compress") {
3072         method = &Benchmark::Compress;
3073       } else if (name == "uncompress") {
3074         method = &Benchmark::Uncompress;
3075 #ifndef ROCKSDB_LITE
3076       } else if (name == "randomtransaction") {
3077         method = &Benchmark::RandomTransaction;
3078         post_process_method = &Benchmark::RandomTransactionVerify;
3079 #endif  // ROCKSDB_LITE
3080       } else if (name == "randomreplacekeys") {
3081         fresh_db = true;
3082         method = &Benchmark::RandomReplaceKeys;
3083       } else if (name == "timeseries") {
3084         timestamp_emulator_.reset(new TimestampEmulator());
3085         if (FLAGS_expire_style == "compaction_filter") {
3086           filter.reset(new ExpiredTimeFilter(timestamp_emulator_));
3087           fprintf(stdout, "Compaction filter is used to remove expired data");
3088           open_options_.compaction_filter = filter.get();
3089         }
3090         fresh_db = true;
3091         method = &Benchmark::TimeSeries;
3092       } else if (name == "stats") {
3093         PrintStats("rocksdb.stats");
3094       } else if (name == "resetstats") {
3095         ResetStats();
3096       } else if (name == "verify") {
3097         VerifyDBFromDB(FLAGS_truth_db);
3098       } else if (name == "levelstats") {
3099         PrintStats("rocksdb.levelstats");
3100       } else if (name == "sstables") {
3101         PrintStats("rocksdb.sstables");
3102       } else if (name == "stats_history") {
3103         PrintStatsHistory();
3104       } else if (name == "replay") {
3105         if (num_threads > 1) {
3106           fprintf(stderr, "Multi-threaded replay is not yet supported\n");
3107           exit(1);
3108         }
3109         if (FLAGS_trace_file == "") {
3110           fprintf(stderr, "Please set --trace_file to be replayed from\n");
3111           exit(1);
3112         }
3113         method = &Benchmark::Replay;
3114       } else if (name == "getmergeoperands") {
3115         method = &Benchmark::GetMergeOperands;
3116       } else if (!name.empty()) {  // No error message for empty name
3117         fprintf(stderr, "unknown benchmark '%s'\n", name.c_str());
3118         exit(1);
3119       }
3120 
3121       if (fresh_db) {
3122         if (FLAGS_use_existing_db) {
3123           fprintf(stdout, "%-12s : skipped (--use_existing_db is true)\n",
3124                   name.c_str());
3125           method = nullptr;
3126         } else {
3127           if (db_.db != nullptr) {
3128             db_.DeleteDBs();
3129             DestroyDB(FLAGS_db, open_options_);
3130           }
3131           Options options = open_options_;
3132           for (size_t i = 0; i < multi_dbs_.size(); i++) {
3133             delete multi_dbs_[i].db;
3134             if (!open_options_.wal_dir.empty()) {
3135               options.wal_dir = GetPathForMultiple(open_options_.wal_dir, i);
3136             }
3137             DestroyDB(GetPathForMultiple(FLAGS_db, i), options);
3138           }
3139           multi_dbs_.clear();
3140         }
3141         Open(&open_options_);  // use open_options for the last accessed
3142       }
3143 
3144       if (method != nullptr) {
3145         fprintf(stdout, "DB path: [%s]\n", FLAGS_db.c_str());
3146 
3147 #ifndef ROCKSDB_LITE
3148         // A trace_file option can be provided both for trace and replay
3149         // operations. But db_bench does not support tracing and replaying at
3150         // the same time, for now. So, start tracing only when it is not a
3151         // replay.
3152         if (FLAGS_trace_file != "" && name != "replay") {
3153           std::unique_ptr<TraceWriter> trace_writer;
3154           Status s = NewFileTraceWriter(FLAGS_env, EnvOptions(),
3155                                         FLAGS_trace_file, &trace_writer);
3156           if (!s.ok()) {
3157             fprintf(stderr, "Encountered an error starting a trace, %s\n",
3158                     s.ToString().c_str());
3159             exit(1);
3160           }
3161           s = db_.db->StartTrace(trace_options_, std::move(trace_writer));
3162           if (!s.ok()) {
3163             fprintf(stderr, "Encountered an error starting a trace, %s\n",
3164                     s.ToString().c_str());
3165             exit(1);
3166           }
3167           fprintf(stdout, "Tracing the workload to: [%s]\n",
3168                   FLAGS_trace_file.c_str());
3169         }
3170         // Start block cache tracing.
3171         if (!FLAGS_block_cache_trace_file.empty()) {
3172           // Sanity checks.
3173           if (FLAGS_block_cache_trace_sampling_frequency <= 0) {
3174             fprintf(stderr,
3175                     "Block cache trace sampling frequency must be higher than "
3176                     "0.\n");
3177             exit(1);
3178           }
3179           if (FLAGS_block_cache_trace_max_trace_file_size_in_bytes <= 0) {
3180             fprintf(stderr,
3181                     "The maximum file size for block cache tracing must be "
3182                     "higher than 0.\n");
3183             exit(1);
3184           }
3185           block_cache_trace_options_.max_trace_file_size =
3186               FLAGS_block_cache_trace_max_trace_file_size_in_bytes;
3187           block_cache_trace_options_.sampling_frequency =
3188               FLAGS_block_cache_trace_sampling_frequency;
3189           std::unique_ptr<TraceWriter> block_cache_trace_writer;
3190           Status s = NewFileTraceWriter(FLAGS_env, EnvOptions(),
3191                                         FLAGS_block_cache_trace_file,
3192                                         &block_cache_trace_writer);
3193           if (!s.ok()) {
3194             fprintf(stderr,
3195                     "Encountered an error when creating trace writer, %s\n",
3196                     s.ToString().c_str());
3197             exit(1);
3198           }
3199           s = db_.db->StartBlockCacheTrace(block_cache_trace_options_,
3200                                            std::move(block_cache_trace_writer));
3201           if (!s.ok()) {
3202             fprintf(
3203                 stderr,
3204                 "Encountered an error when starting block cache tracing, %s\n",
3205                 s.ToString().c_str());
3206             exit(1);
3207           }
3208           fprintf(stdout, "Tracing block cache accesses to: [%s]\n",
3209                   FLAGS_block_cache_trace_file.c_str());
3210         }
3211 #endif  // ROCKSDB_LITE
3212 
3213         if (num_warmup > 0) {
3214           printf("Warming up benchmark by running %d times\n", num_warmup);
3215         }
3216 
3217         for (int i = 0; i < num_warmup; i++) {
3218           RunBenchmark(num_threads, name, method);
3219         }
3220 
3221         if (num_repeat > 1) {
3222           printf("Running benchmark for %d times\n", num_repeat);
3223         }
3224 
3225         CombinedStats combined_stats;
3226         for (int i = 0; i < num_repeat; i++) {
3227           Stats stats = RunBenchmark(num_threads, name, method);
3228           combined_stats.AddStats(stats);
3229         }
3230         if (num_repeat > 1) {
3231           combined_stats.Report(name);
3232         }
3233       }
3234       if (post_process_method != nullptr) {
3235         (this->*post_process_method)();
3236       }
3237     }
3238 
3239     if (secondary_update_thread_) {
3240       secondary_update_stopped_.store(1, std::memory_order_relaxed);
3241       secondary_update_thread_->join();
3242       secondary_update_thread_.reset();
3243     }
3244 
3245 #ifndef ROCKSDB_LITE
3246     if (name != "replay" && FLAGS_trace_file != "") {
3247       Status s = db_.db->EndTrace();
3248       if (!s.ok()) {
3249         fprintf(stderr, "Encountered an error ending the trace, %s\n",
3250                 s.ToString().c_str());
3251       }
3252     }
3253     if (!FLAGS_block_cache_trace_file.empty()) {
3254       Status s = db_.db->EndBlockCacheTrace();
3255       if (!s.ok()) {
3256         fprintf(stderr,
3257                 "Encountered an error ending the block cache tracing, %s\n",
3258                 s.ToString().c_str());
3259       }
3260     }
3261 #endif  // ROCKSDB_LITE
3262 
3263     if (FLAGS_statistics) {
3264       fprintf(stdout, "STATISTICS:\n%s\n", dbstats->ToString().c_str());
3265     }
3266     if (FLAGS_simcache_size >= 0) {
3267       fprintf(stdout, "SIMULATOR CACHE STATISTICS:\n%s\n",
3268               static_cast_with_check<SimCache, Cache>(cache_.get())
3269                   ->ToString()
3270                   .c_str());
3271     }
3272 
3273 #ifndef ROCKSDB_LITE
3274     if (FLAGS_use_secondary_db) {
3275       fprintf(stdout, "Secondary instance updated  %" PRIu64 " times.\n",
3276               secondary_db_updates_);
3277     }
3278 #endif  // ROCKSDB_LITE
3279   }
3280 
3281  private:
3282   std::shared_ptr<TimestampEmulator> timestamp_emulator_;
3283   std::unique_ptr<port::Thread> secondary_update_thread_;
3284   std::atomic<int> secondary_update_stopped_{0};
3285 #ifndef ROCKSDB_LITE
3286   uint64_t secondary_db_updates_ = 0;
3287 #endif  // ROCKSDB_LITE
3288   struct ThreadArg {
3289     Benchmark* bm;
3290     SharedState* shared;
3291     ThreadState* thread;
3292     void (Benchmark::*method)(ThreadState*);
3293   };
3294 
ThreadBody(void * v)3295   static void ThreadBody(void* v) {
3296     ThreadArg* arg = reinterpret_cast<ThreadArg*>(v);
3297     SharedState* shared = arg->shared;
3298     ThreadState* thread = arg->thread;
3299     {
3300       MutexLock l(&shared->mu);
3301       shared->num_initialized++;
3302       if (shared->num_initialized >= shared->total) {
3303         shared->cv.SignalAll();
3304       }
3305       while (!shared->start) {
3306         shared->cv.Wait();
3307       }
3308     }
3309 
3310     SetPerfLevel(static_cast<PerfLevel> (shared->perf_level));
3311     perf_context.EnablePerLevelPerfContext();
3312     thread->stats.Start(thread->tid);
3313     (arg->bm->*(arg->method))(thread);
3314     thread->stats.Stop();
3315 
3316     {
3317       MutexLock l(&shared->mu);
3318       shared->num_done++;
3319       if (shared->num_done >= shared->total) {
3320         shared->cv.SignalAll();
3321       }
3322     }
3323   }
3324 
RunBenchmark(int n,Slice name,void (Benchmark::* method)(ThreadState *))3325   Stats RunBenchmark(int n, Slice name,
3326                      void (Benchmark::*method)(ThreadState*)) {
3327     SharedState shared;
3328     shared.total = n;
3329     shared.num_initialized = 0;
3330     shared.num_done = 0;
3331     shared.start = false;
3332     if (FLAGS_benchmark_write_rate_limit > 0) {
3333       shared.write_rate_limiter.reset(
3334           NewGenericRateLimiter(FLAGS_benchmark_write_rate_limit));
3335     }
3336     if (FLAGS_benchmark_read_rate_limit > 0) {
3337       shared.read_rate_limiter.reset(NewGenericRateLimiter(
3338           FLAGS_benchmark_read_rate_limit, 100000 /* refill_period_us */,
3339           10 /* fairness */, RateLimiter::Mode::kReadsOnly));
3340     }
3341 
3342     std::unique_ptr<ReporterAgent> reporter_agent;
3343     if (FLAGS_report_interval_seconds > 0) {
3344       reporter_agent.reset(new ReporterAgent(FLAGS_env, FLAGS_report_file,
3345                                              FLAGS_report_interval_seconds));
3346     }
3347 
3348     ThreadArg* arg = new ThreadArg[n];
3349 
3350     for (int i = 0; i < n; i++) {
3351 #ifdef NUMA
3352       if (FLAGS_enable_numa) {
3353         // Performs a local allocation of memory to threads in numa node.
3354         int n_nodes = numa_num_task_nodes();  // Number of nodes in NUMA.
3355         numa_exit_on_error = 1;
3356         int numa_node = i % n_nodes;
3357         bitmask* nodes = numa_allocate_nodemask();
3358         numa_bitmask_clearall(nodes);
3359         numa_bitmask_setbit(nodes, numa_node);
3360         // numa_bind() call binds the process to the node and these
3361         // properties are passed on to the thread that is created in
3362         // StartThread method called later in the loop.
3363         numa_bind(nodes);
3364         numa_set_strict(1);
3365         numa_free_nodemask(nodes);
3366       }
3367 #endif
3368       arg[i].bm = this;
3369       arg[i].method = method;
3370       arg[i].shared = &shared;
3371       arg[i].thread = new ThreadState(i);
3372       arg[i].thread->stats.SetReporterAgent(reporter_agent.get());
3373       arg[i].thread->shared = &shared;
3374       FLAGS_env->StartThread(ThreadBody, &arg[i]);
3375     }
3376 
3377     shared.mu.Lock();
3378     while (shared.num_initialized < n) {
3379       shared.cv.Wait();
3380     }
3381 
3382     shared.start = true;
3383     shared.cv.SignalAll();
3384     while (shared.num_done < n) {
3385       shared.cv.Wait();
3386     }
3387     shared.mu.Unlock();
3388 
3389     // Stats for some threads can be excluded.
3390     Stats merge_stats;
3391     for (int i = 0; i < n; i++) {
3392       merge_stats.Merge(arg[i].thread->stats);
3393     }
3394     merge_stats.Report(name);
3395 
3396     for (int i = 0; i < n; i++) {
3397       delete arg[i].thread;
3398     }
3399     delete[] arg;
3400 
3401     return merge_stats;
3402   }
3403 
Crc32c(ThreadState * thread)3404   void Crc32c(ThreadState* thread) {
3405     // Checksum about 500MB of data total
3406     const int size = FLAGS_block_size; // use --block_size option for db_bench
3407     std::string labels = "(" + ToString(FLAGS_block_size) + " per op)";
3408     const char* label = labels.c_str();
3409 
3410     std::string data(size, 'x');
3411     int64_t bytes = 0;
3412     uint32_t crc = 0;
3413     while (bytes < 500 * 1048576) {
3414       crc = crc32c::Value(data.data(), size);
3415       thread->stats.FinishedOps(nullptr, nullptr, 1, kCrc);
3416       bytes += size;
3417     }
3418     // Print so result is not dead
3419     fprintf(stderr, "... crc=0x%x\r", static_cast<unsigned int>(crc));
3420 
3421     thread->stats.AddBytes(bytes);
3422     thread->stats.AddMessage(label);
3423   }
3424 
xxHash(ThreadState * thread)3425   void xxHash(ThreadState* thread) {
3426     // Checksum about 500MB of data total
3427     const int size = 4096;
3428     const char* label = "(4K per op)";
3429     std::string data(size, 'x');
3430     int64_t bytes = 0;
3431     unsigned int xxh32 = 0;
3432     while (bytes < 500 * 1048576) {
3433       xxh32 = XXH32(data.data(), size, 0);
3434       thread->stats.FinishedOps(nullptr, nullptr, 1, kHash);
3435       bytes += size;
3436     }
3437     // Print so result is not dead
3438     fprintf(stderr, "... xxh32=0x%x\r", static_cast<unsigned int>(xxh32));
3439 
3440     thread->stats.AddBytes(bytes);
3441     thread->stats.AddMessage(label);
3442   }
3443 
AcquireLoad(ThreadState * thread)3444   void AcquireLoad(ThreadState* thread) {
3445     int dummy;
3446     std::atomic<void*> ap(&dummy);
3447     int count = 0;
3448     void *ptr = nullptr;
3449     thread->stats.AddMessage("(each op is 1000 loads)");
3450     while (count < 100000) {
3451       for (int i = 0; i < 1000; i++) {
3452         ptr = ap.load(std::memory_order_acquire);
3453       }
3454       count++;
3455       thread->stats.FinishedOps(nullptr, nullptr, 1, kOthers);
3456     }
3457     if (ptr == nullptr) exit(1);  // Disable unused variable warning.
3458   }
3459 
Compress(ThreadState * thread)3460   void Compress(ThreadState *thread) {
3461     RandomGenerator gen;
3462     Slice input = gen.Generate(FLAGS_block_size);
3463     int64_t bytes = 0;
3464     int64_t produced = 0;
3465     bool ok = true;
3466     std::string compressed;
3467     CompressionOptions opts;
3468     CompressionContext context(FLAGS_compression_type_e);
3469     CompressionInfo info(opts, context, CompressionDict::GetEmptyDict(),
3470                          FLAGS_compression_type_e,
3471                          FLAGS_sample_for_compression);
3472     // Compress 1G
3473     while (ok && bytes < int64_t(1) << 30) {
3474       compressed.clear();
3475       ok = CompressSlice(info, input, &compressed);
3476       produced += compressed.size();
3477       bytes += input.size();
3478       thread->stats.FinishedOps(nullptr, nullptr, 1, kCompress);
3479     }
3480 
3481     if (!ok) {
3482       thread->stats.AddMessage("(compression failure)");
3483     } else {
3484       char buf[340];
3485       snprintf(buf, sizeof(buf), "(output: %.1f%%)",
3486                (produced * 100.0) / bytes);
3487       thread->stats.AddMessage(buf);
3488       thread->stats.AddBytes(bytes);
3489     }
3490   }
3491 
Uncompress(ThreadState * thread)3492   void Uncompress(ThreadState *thread) {
3493     RandomGenerator gen;
3494     Slice input = gen.Generate(FLAGS_block_size);
3495     std::string compressed;
3496 
3497     CompressionContext compression_ctx(FLAGS_compression_type_e);
3498     CompressionOptions compression_opts;
3499     CompressionInfo compression_info(
3500         compression_opts, compression_ctx, CompressionDict::GetEmptyDict(),
3501         FLAGS_compression_type_e, FLAGS_sample_for_compression);
3502     UncompressionContext uncompression_ctx(FLAGS_compression_type_e);
3503     UncompressionInfo uncompression_info(uncompression_ctx,
3504                                          UncompressionDict::GetEmptyDict(),
3505                                          FLAGS_compression_type_e);
3506 
3507     bool ok = CompressSlice(compression_info, input, &compressed);
3508     int64_t bytes = 0;
3509     int decompress_size;
3510     while (ok && bytes < 1024 * 1048576) {
3511       CacheAllocationPtr uncompressed;
3512       switch (FLAGS_compression_type_e) {
3513         case ROCKSDB_NAMESPACE::kSnappyCompression: {
3514           // get size and allocate here to make comparison fair
3515           size_t ulength = 0;
3516           if (!Snappy_GetUncompressedLength(compressed.data(),
3517                                             compressed.size(), &ulength)) {
3518             ok = false;
3519             break;
3520           }
3521           uncompressed = AllocateBlock(ulength, nullptr);
3522           ok = Snappy_Uncompress(compressed.data(), compressed.size(),
3523                                  uncompressed.get());
3524           break;
3525         }
3526         case ROCKSDB_NAMESPACE::kZlibCompression:
3527           uncompressed =
3528               Zlib_Uncompress(uncompression_info, compressed.data(),
3529                               compressed.size(), &decompress_size, 2);
3530           ok = uncompressed.get() != nullptr;
3531           break;
3532         case ROCKSDB_NAMESPACE::kBZip2Compression:
3533           uncompressed = BZip2_Uncompress(compressed.data(), compressed.size(),
3534                                           &decompress_size, 2);
3535           ok = uncompressed.get() != nullptr;
3536           break;
3537         case ROCKSDB_NAMESPACE::kLZ4Compression:
3538           uncompressed = LZ4_Uncompress(uncompression_info, compressed.data(),
3539                                         compressed.size(), &decompress_size, 2);
3540           ok = uncompressed.get() != nullptr;
3541           break;
3542         case ROCKSDB_NAMESPACE::kLZ4HCCompression:
3543           uncompressed = LZ4_Uncompress(uncompression_info, compressed.data(),
3544                                         compressed.size(), &decompress_size, 2);
3545           ok = uncompressed.get() != nullptr;
3546           break;
3547         case ROCKSDB_NAMESPACE::kXpressCompression:
3548           uncompressed.reset(XPRESS_Uncompress(
3549               compressed.data(), compressed.size(), &decompress_size));
3550           ok = uncompressed.get() != nullptr;
3551           break;
3552         case ROCKSDB_NAMESPACE::kZSTD:
3553           uncompressed = ZSTD_Uncompress(uncompression_info, compressed.data(),
3554                                          compressed.size(), &decompress_size);
3555           ok = uncompressed.get() != nullptr;
3556           break;
3557         default:
3558           ok = false;
3559       }
3560       bytes += input.size();
3561       thread->stats.FinishedOps(nullptr, nullptr, 1, kUncompress);
3562     }
3563 
3564     if (!ok) {
3565       thread->stats.AddMessage("(compression failure)");
3566     } else {
3567       thread->stats.AddBytes(bytes);
3568     }
3569   }
3570 
3571   // Returns true if the options is initialized from the specified
3572   // options file.
InitializeOptionsFromFile(Options * opts)3573   bool InitializeOptionsFromFile(Options* opts) {
3574 #ifndef ROCKSDB_LITE
3575     printf("Initializing RocksDB Options from the specified file\n");
3576     DBOptions db_opts;
3577     std::vector<ColumnFamilyDescriptor> cf_descs;
3578     if (FLAGS_options_file != "") {
3579       auto s = LoadOptionsFromFile(FLAGS_options_file, FLAGS_env, &db_opts,
3580                                    &cf_descs);
3581       db_opts.env = FLAGS_env;
3582       if (s.ok()) {
3583         *opts = Options(db_opts, cf_descs[0].options);
3584         return true;
3585       }
3586       fprintf(stderr, "Unable to load options file %s --- %s\n",
3587               FLAGS_options_file.c_str(), s.ToString().c_str());
3588       exit(1);
3589     }
3590 #else
3591     (void)opts;
3592 #endif
3593     return false;
3594   }
3595 
InitializeOptionsFromFlags(Options * opts)3596   void InitializeOptionsFromFlags(Options* opts) {
3597     printf("Initializing RocksDB Options from command-line flags\n");
3598     Options& options = *opts;
3599 
3600     assert(db_.db == nullptr);
3601 
3602     options.env = FLAGS_env;
3603     options.max_open_files = FLAGS_open_files;
3604     if (FLAGS_cost_write_buffer_to_cache || FLAGS_db_write_buffer_size != 0) {
3605       options.write_buffer_manager.reset(
3606           new WriteBufferManager(FLAGS_db_write_buffer_size, cache_));
3607     }
3608     options.write_buffer_size = FLAGS_write_buffer_size;
3609     options.max_write_buffer_number = FLAGS_max_write_buffer_number;
3610     options.min_write_buffer_number_to_merge =
3611       FLAGS_min_write_buffer_number_to_merge;
3612     options.max_write_buffer_number_to_maintain =
3613         FLAGS_max_write_buffer_number_to_maintain;
3614     options.max_write_buffer_size_to_maintain =
3615         FLAGS_max_write_buffer_size_to_maintain;
3616     options.max_background_jobs = FLAGS_max_background_jobs;
3617     options.max_background_compactions = FLAGS_max_background_compactions;
3618     options.max_subcompactions = static_cast<uint32_t>(FLAGS_subcompactions);
3619     options.max_background_flushes = FLAGS_max_background_flushes;
3620     options.compaction_style = FLAGS_compaction_style_e;
3621     options.compaction_pri = FLAGS_compaction_pri_e;
3622     options.allow_mmap_reads = FLAGS_mmap_read;
3623     options.allow_mmap_writes = FLAGS_mmap_write;
3624     options.use_direct_reads = FLAGS_use_direct_reads;
3625     options.use_direct_io_for_flush_and_compaction =
3626         FLAGS_use_direct_io_for_flush_and_compaction;
3627 #ifndef ROCKSDB_LITE
3628     options.ttl = FLAGS_fifo_compaction_ttl;
3629     options.compaction_options_fifo = CompactionOptionsFIFO(
3630         FLAGS_fifo_compaction_max_table_files_size_mb * 1024 * 1024,
3631         FLAGS_fifo_compaction_allow_compaction);
3632 #endif  // ROCKSDB_LITE
3633     if (FLAGS_prefix_size != 0) {
3634       options.prefix_extractor.reset(
3635           NewFixedPrefixTransform(FLAGS_prefix_size));
3636     }
3637     if (FLAGS_use_uint64_comparator) {
3638       options.comparator = test::Uint64Comparator();
3639       if (FLAGS_key_size != 8) {
3640         fprintf(stderr, "Using Uint64 comparator but key size is not 8.\n");
3641         exit(1);
3642       }
3643     }
3644     if (FLAGS_use_stderr_info_logger) {
3645       options.info_log.reset(new StderrLogger());
3646     }
3647     options.memtable_huge_page_size = FLAGS_memtable_use_huge_page ? 2048 : 0;
3648     options.memtable_prefix_bloom_size_ratio = FLAGS_memtable_bloom_size_ratio;
3649     options.memtable_whole_key_filtering = FLAGS_memtable_whole_key_filtering;
3650     if (FLAGS_memtable_insert_with_hint_prefix_size > 0) {
3651       options.memtable_insert_with_hint_prefix_extractor.reset(
3652           NewCappedPrefixTransform(
3653               FLAGS_memtable_insert_with_hint_prefix_size));
3654     }
3655     options.bloom_locality = FLAGS_bloom_locality;
3656     options.max_file_opening_threads = FLAGS_file_opening_threads;
3657     options.new_table_reader_for_compaction_inputs =
3658         FLAGS_new_table_reader_for_compaction_inputs;
3659     options.compaction_readahead_size = FLAGS_compaction_readahead_size;
3660     options.log_readahead_size = FLAGS_log_readahead_size;
3661     options.random_access_max_buffer_size = FLAGS_random_access_max_buffer_size;
3662     options.writable_file_max_buffer_size = FLAGS_writable_file_max_buffer_size;
3663     options.use_fsync = FLAGS_use_fsync;
3664     options.num_levels = FLAGS_num_levels;
3665     options.target_file_size_base = FLAGS_target_file_size_base;
3666     options.target_file_size_multiplier = FLAGS_target_file_size_multiplier;
3667     options.max_bytes_for_level_base = FLAGS_max_bytes_for_level_base;
3668     options.level_compaction_dynamic_level_bytes =
3669         FLAGS_level_compaction_dynamic_level_bytes;
3670     options.max_bytes_for_level_multiplier =
3671         FLAGS_max_bytes_for_level_multiplier;
3672     if ((FLAGS_prefix_size == 0) && (FLAGS_rep_factory == kPrefixHash ||
3673                                      FLAGS_rep_factory == kHashLinkedList)) {
3674       fprintf(stderr, "prefix_size should be non-zero if PrefixHash or "
3675                       "HashLinkedList memtablerep is used\n");
3676       exit(1);
3677     }
3678     switch (FLAGS_rep_factory) {
3679       case kSkipList:
3680         options.memtable_factory.reset(new SkipListFactory(
3681             FLAGS_skip_list_lookahead));
3682         break;
3683 #ifndef ROCKSDB_LITE
3684       case kPrefixHash:
3685         options.memtable_factory.reset(
3686             NewHashSkipListRepFactory(FLAGS_hash_bucket_count));
3687         break;
3688       case kHashLinkedList:
3689         options.memtable_factory.reset(NewHashLinkListRepFactory(
3690             FLAGS_hash_bucket_count));
3691         break;
3692       case kVectorRep:
3693         options.memtable_factory.reset(
3694           new VectorRepFactory
3695         );
3696         break;
3697 #else
3698       default:
3699         fprintf(stderr, "Only skip list is supported in lite mode\n");
3700         exit(1);
3701 #endif  // ROCKSDB_LITE
3702     }
3703     if (FLAGS_use_plain_table) {
3704 #ifndef ROCKSDB_LITE
3705       if (FLAGS_rep_factory != kPrefixHash &&
3706           FLAGS_rep_factory != kHashLinkedList) {
3707         fprintf(stderr, "Waring: plain table is used with skipList\n");
3708       }
3709 
3710       int bloom_bits_per_key = FLAGS_bloom_bits;
3711       if (bloom_bits_per_key < 0) {
3712         bloom_bits_per_key = 0;
3713       }
3714 
3715       PlainTableOptions plain_table_options;
3716       plain_table_options.user_key_len = FLAGS_key_size;
3717       plain_table_options.bloom_bits_per_key = bloom_bits_per_key;
3718       plain_table_options.hash_table_ratio = 0.75;
3719       options.table_factory = std::shared_ptr<TableFactory>(
3720           NewPlainTableFactory(plain_table_options));
3721 #else
3722       fprintf(stderr, "Plain table is not supported in lite mode\n");
3723       exit(1);
3724 #endif  // ROCKSDB_LITE
3725     } else if (FLAGS_use_cuckoo_table) {
3726 #ifndef ROCKSDB_LITE
3727       if (FLAGS_cuckoo_hash_ratio > 1 || FLAGS_cuckoo_hash_ratio < 0) {
3728         fprintf(stderr, "Invalid cuckoo_hash_ratio\n");
3729         exit(1);
3730       }
3731 
3732       if (!FLAGS_mmap_read) {
3733         fprintf(stderr, "cuckoo table format requires mmap read to operate\n");
3734         exit(1);
3735       }
3736 
3737       ROCKSDB_NAMESPACE::CuckooTableOptions table_options;
3738       table_options.hash_table_ratio = FLAGS_cuckoo_hash_ratio;
3739       table_options.identity_as_first_hash = FLAGS_identity_as_first_hash;
3740       options.table_factory = std::shared_ptr<TableFactory>(
3741           NewCuckooTableFactory(table_options));
3742 #else
3743       fprintf(stderr, "Cuckoo table is not supported in lite mode\n");
3744       exit(1);
3745 #endif  // ROCKSDB_LITE
3746     } else {
3747       BlockBasedTableOptions block_based_options;
3748       if (FLAGS_use_hash_search) {
3749         if (FLAGS_prefix_size == 0) {
3750           fprintf(stderr,
3751               "prefix_size not assigned when enable use_hash_search \n");
3752           exit(1);
3753         }
3754         block_based_options.index_type = BlockBasedTableOptions::kHashSearch;
3755       } else {
3756         block_based_options.index_type = BlockBasedTableOptions::kBinarySearch;
3757       }
3758       if (FLAGS_partition_index_and_filters || FLAGS_partition_index) {
3759         if (FLAGS_index_with_first_key) {
3760           fprintf(stderr,
3761                   "--index_with_first_key is not compatible with"
3762                   " partition index.");
3763         }
3764         if (FLAGS_use_hash_search) {
3765           fprintf(stderr,
3766                   "use_hash_search is incompatible with "
3767                   "partition index and is ignored");
3768         }
3769         block_based_options.index_type =
3770             BlockBasedTableOptions::kTwoLevelIndexSearch;
3771         block_based_options.metadata_block_size = FLAGS_metadata_block_size;
3772         if (FLAGS_partition_index_and_filters) {
3773           block_based_options.partition_filters = true;
3774         }
3775       } else if (FLAGS_index_with_first_key) {
3776         block_based_options.index_type =
3777             BlockBasedTableOptions::kBinarySearchWithFirstKey;
3778       }
3779       BlockBasedTableOptions::IndexShorteningMode index_shortening =
3780           block_based_options.index_shortening;
3781       switch (FLAGS_index_shortening_mode) {
3782         case 0:
3783           index_shortening =
3784               BlockBasedTableOptions::IndexShorteningMode::kNoShortening;
3785           break;
3786         case 1:
3787           index_shortening =
3788               BlockBasedTableOptions::IndexShorteningMode::kShortenSeparators;
3789           break;
3790         case 2:
3791           index_shortening = BlockBasedTableOptions::IndexShorteningMode::
3792               kShortenSeparatorsAndSuccessor;
3793           break;
3794         default:
3795           fprintf(stderr, "Unknown key shortening mode\n");
3796       }
3797       block_based_options.index_shortening = index_shortening;
3798       if (cache_ == nullptr) {
3799         block_based_options.no_block_cache = true;
3800       }
3801       block_based_options.cache_index_and_filter_blocks =
3802           FLAGS_cache_index_and_filter_blocks;
3803       block_based_options.pin_l0_filter_and_index_blocks_in_cache =
3804           FLAGS_pin_l0_filter_and_index_blocks_in_cache;
3805       block_based_options.pin_top_level_index_and_filter =
3806           FLAGS_pin_top_level_index_and_filter;
3807       if (FLAGS_cache_high_pri_pool_ratio > 1e-6) {  // > 0.0 + eps
3808         block_based_options.cache_index_and_filter_blocks_with_high_priority =
3809             true;
3810       }
3811       block_based_options.block_cache = cache_;
3812       block_based_options.block_cache_compressed = compressed_cache_;
3813       block_based_options.block_size = FLAGS_block_size;
3814       block_based_options.block_restart_interval = FLAGS_block_restart_interval;
3815       block_based_options.index_block_restart_interval =
3816           FLAGS_index_block_restart_interval;
3817       block_based_options.filter_policy = filter_policy_;
3818       block_based_options.format_version =
3819           static_cast<uint32_t>(FLAGS_format_version);
3820       block_based_options.read_amp_bytes_per_bit = FLAGS_read_amp_bytes_per_bit;
3821       block_based_options.enable_index_compression =
3822           FLAGS_enable_index_compression;
3823       block_based_options.block_align = FLAGS_block_align;
3824       if (FLAGS_use_data_block_hash_index) {
3825         block_based_options.data_block_index_type =
3826             ROCKSDB_NAMESPACE::BlockBasedTableOptions::kDataBlockBinaryAndHash;
3827       } else {
3828         block_based_options.data_block_index_type =
3829             ROCKSDB_NAMESPACE::BlockBasedTableOptions::kDataBlockBinarySearch;
3830       }
3831       block_based_options.data_block_hash_table_util_ratio =
3832           FLAGS_data_block_hash_table_util_ratio;
3833       if (FLAGS_read_cache_path != "") {
3834 #ifndef ROCKSDB_LITE
3835         Status rc_status;
3836 
3837         // Read cache need to be provided with a the Logger, we will put all
3838         // reac cache logs in the read cache path in a file named rc_LOG
3839         rc_status = FLAGS_env->CreateDirIfMissing(FLAGS_read_cache_path);
3840         std::shared_ptr<Logger> read_cache_logger;
3841         if (rc_status.ok()) {
3842           rc_status = FLAGS_env->NewLogger(FLAGS_read_cache_path + "/rc_LOG",
3843                                            &read_cache_logger);
3844         }
3845 
3846         if (rc_status.ok()) {
3847           PersistentCacheConfig rc_cfg(FLAGS_env, FLAGS_read_cache_path,
3848                                        FLAGS_read_cache_size,
3849                                        read_cache_logger);
3850 
3851           rc_cfg.enable_direct_reads = FLAGS_read_cache_direct_read;
3852           rc_cfg.enable_direct_writes = FLAGS_read_cache_direct_write;
3853           rc_cfg.writer_qdepth = 4;
3854           rc_cfg.writer_dispatch_size = 4 * 1024;
3855 
3856           auto pcache = std::make_shared<BlockCacheTier>(rc_cfg);
3857           block_based_options.persistent_cache = pcache;
3858           rc_status = pcache->Open();
3859         }
3860 
3861         if (!rc_status.ok()) {
3862           fprintf(stderr, "Error initializing read cache, %s\n",
3863                   rc_status.ToString().c_str());
3864           exit(1);
3865         }
3866 #else
3867         fprintf(stderr, "Read cache is not supported in LITE\n");
3868         exit(1);
3869 
3870 #endif
3871       }
3872       options.table_factory.reset(
3873           NewBlockBasedTableFactory(block_based_options));
3874     }
3875     if (FLAGS_max_bytes_for_level_multiplier_additional_v.size() > 0) {
3876       if (FLAGS_max_bytes_for_level_multiplier_additional_v.size() !=
3877           static_cast<unsigned int>(FLAGS_num_levels)) {
3878         fprintf(stderr, "Insufficient number of fanouts specified %d\n",
3879                 static_cast<int>(
3880                     FLAGS_max_bytes_for_level_multiplier_additional_v.size()));
3881         exit(1);
3882       }
3883       options.max_bytes_for_level_multiplier_additional =
3884         FLAGS_max_bytes_for_level_multiplier_additional_v;
3885     }
3886     options.level0_stop_writes_trigger = FLAGS_level0_stop_writes_trigger;
3887     options.level0_file_num_compaction_trigger =
3888         FLAGS_level0_file_num_compaction_trigger;
3889     options.level0_slowdown_writes_trigger =
3890       FLAGS_level0_slowdown_writes_trigger;
3891     options.compression = FLAGS_compression_type_e;
3892     options.sample_for_compression = FLAGS_sample_for_compression;
3893     options.WAL_ttl_seconds = FLAGS_wal_ttl_seconds;
3894     options.WAL_size_limit_MB = FLAGS_wal_size_limit_MB;
3895     options.max_total_wal_size = FLAGS_max_total_wal_size;
3896 
3897     if (FLAGS_min_level_to_compress >= 0) {
3898       assert(FLAGS_min_level_to_compress <= FLAGS_num_levels);
3899       options.compression_per_level.resize(FLAGS_num_levels);
3900       for (int i = 0; i < FLAGS_min_level_to_compress; i++) {
3901         options.compression_per_level[i] = kNoCompression;
3902       }
3903       for (int i = FLAGS_min_level_to_compress;
3904            i < FLAGS_num_levels; i++) {
3905         options.compression_per_level[i] = FLAGS_compression_type_e;
3906       }
3907     }
3908     options.soft_rate_limit = FLAGS_soft_rate_limit;
3909     options.hard_rate_limit = FLAGS_hard_rate_limit;
3910     options.soft_pending_compaction_bytes_limit =
3911         FLAGS_soft_pending_compaction_bytes_limit;
3912     options.hard_pending_compaction_bytes_limit =
3913         FLAGS_hard_pending_compaction_bytes_limit;
3914     options.delayed_write_rate = FLAGS_delayed_write_rate;
3915     options.allow_concurrent_memtable_write =
3916         FLAGS_allow_concurrent_memtable_write;
3917     options.inplace_update_support = FLAGS_inplace_update_support;
3918     options.inplace_update_num_locks = FLAGS_inplace_update_num_locks;
3919     options.enable_write_thread_adaptive_yield =
3920         FLAGS_enable_write_thread_adaptive_yield;
3921     options.enable_pipelined_write = FLAGS_enable_pipelined_write;
3922     options.unordered_write = FLAGS_unordered_write;
3923     options.write_thread_max_yield_usec = FLAGS_write_thread_max_yield_usec;
3924     options.write_thread_slow_yield_usec = FLAGS_write_thread_slow_yield_usec;
3925     options.rate_limit_delay_max_milliseconds =
3926       FLAGS_rate_limit_delay_max_milliseconds;
3927     options.table_cache_numshardbits = FLAGS_table_cache_numshardbits;
3928     options.max_compaction_bytes = FLAGS_max_compaction_bytes;
3929     options.disable_auto_compactions = FLAGS_disable_auto_compactions;
3930     options.optimize_filters_for_hits = FLAGS_optimize_filters_for_hits;
3931 
3932     // fill storage options
3933     options.advise_random_on_open = FLAGS_advise_random_on_open;
3934     options.access_hint_on_compaction_start = FLAGS_compaction_fadvice_e;
3935     options.use_adaptive_mutex = FLAGS_use_adaptive_mutex;
3936     options.bytes_per_sync = FLAGS_bytes_per_sync;
3937     options.wal_bytes_per_sync = FLAGS_wal_bytes_per_sync;
3938 
3939     // merge operator options
3940     options.merge_operator = MergeOperators::CreateFromStringId(
3941         FLAGS_merge_operator);
3942     if (options.merge_operator == nullptr && !FLAGS_merge_operator.empty()) {
3943       fprintf(stderr, "invalid merge operator: %s\n",
3944               FLAGS_merge_operator.c_str());
3945       exit(1);
3946     }
3947     options.max_successive_merges = FLAGS_max_successive_merges;
3948     options.report_bg_io_stats = FLAGS_report_bg_io_stats;
3949 
3950     // set universal style compaction configurations, if applicable
3951     if (FLAGS_universal_size_ratio != 0) {
3952       options.compaction_options_universal.size_ratio =
3953         FLAGS_universal_size_ratio;
3954     }
3955     if (FLAGS_universal_min_merge_width != 0) {
3956       options.compaction_options_universal.min_merge_width =
3957         FLAGS_universal_min_merge_width;
3958     }
3959     if (FLAGS_universal_max_merge_width != 0) {
3960       options.compaction_options_universal.max_merge_width =
3961         FLAGS_universal_max_merge_width;
3962     }
3963     if (FLAGS_universal_max_size_amplification_percent != 0) {
3964       options.compaction_options_universal.max_size_amplification_percent =
3965         FLAGS_universal_max_size_amplification_percent;
3966     }
3967     if (FLAGS_universal_compression_size_percent != -1) {
3968       options.compaction_options_universal.compression_size_percent =
3969         FLAGS_universal_compression_size_percent;
3970     }
3971     options.compaction_options_universal.allow_trivial_move =
3972         FLAGS_universal_allow_trivial_move;
3973     if (FLAGS_thread_status_per_interval > 0) {
3974       options.enable_thread_tracking = true;
3975     }
3976 
3977 #ifndef ROCKSDB_LITE
3978     if (FLAGS_readonly && FLAGS_transaction_db) {
3979       fprintf(stderr, "Cannot use readonly flag with transaction_db\n");
3980       exit(1);
3981     }
3982     if (FLAGS_use_secondary_db &&
3983         (FLAGS_transaction_db || FLAGS_optimistic_transaction_db)) {
3984       fprintf(stderr, "Cannot use use_secondary_db flag with transaction_db\n");
3985       exit(1);
3986     }
3987 #endif  // ROCKSDB_LITE
3988 
3989   }
3990 
InitializeOptionsGeneral(Options * opts)3991   void InitializeOptionsGeneral(Options* opts) {
3992     Options& options = *opts;
3993 
3994     options.create_missing_column_families = FLAGS_num_column_families > 1;
3995     options.statistics = dbstats;
3996     options.wal_dir = FLAGS_wal_dir;
3997     options.create_if_missing = !FLAGS_use_existing_db;
3998     options.dump_malloc_stats = FLAGS_dump_malloc_stats;
3999     options.stats_dump_period_sec =
4000         static_cast<unsigned int>(FLAGS_stats_dump_period_sec);
4001     options.stats_persist_period_sec =
4002         static_cast<unsigned int>(FLAGS_stats_persist_period_sec);
4003     options.persist_stats_to_disk = FLAGS_persist_stats_to_disk;
4004     options.stats_history_buffer_size =
4005         static_cast<size_t>(FLAGS_stats_history_buffer_size);
4006 
4007     options.compression_opts.level = FLAGS_compression_level;
4008     options.compression_opts.max_dict_bytes = FLAGS_compression_max_dict_bytes;
4009     options.compression_opts.zstd_max_train_bytes =
4010         FLAGS_compression_zstd_max_train_bytes;
4011     // If this is a block based table, set some related options
4012     if (options.table_factory->Name() == BlockBasedTableFactory::kName &&
4013         options.table_factory->GetOptions() != nullptr) {
4014       BlockBasedTableOptions* table_options =
4015           reinterpret_cast<BlockBasedTableOptions*>(
4016               options.table_factory->GetOptions());
4017       if (FLAGS_cache_size) {
4018         table_options->block_cache = cache_;
4019       }
4020       if (FLAGS_bloom_bits >= 0) {
4021         table_options->filter_policy.reset(NewBloomFilterPolicy(
4022             FLAGS_bloom_bits, FLAGS_use_block_based_filter));
4023       }
4024     }
4025     if (FLAGS_row_cache_size) {
4026       if (FLAGS_cache_numshardbits >= 1) {
4027         options.row_cache =
4028             NewLRUCache(FLAGS_row_cache_size, FLAGS_cache_numshardbits);
4029       } else {
4030         options.row_cache = NewLRUCache(FLAGS_row_cache_size);
4031       }
4032     }
4033     if (FLAGS_enable_io_prio) {
4034       FLAGS_env->LowerThreadPoolIOPriority(Env::LOW);
4035       FLAGS_env->LowerThreadPoolIOPriority(Env::HIGH);
4036     }
4037     if (FLAGS_enable_cpu_prio) {
4038       FLAGS_env->LowerThreadPoolCPUPriority(Env::LOW);
4039       FLAGS_env->LowerThreadPoolCPUPriority(Env::HIGH);
4040     }
4041     options.env = FLAGS_env;
4042     if (FLAGS_sine_write_rate) {
4043       FLAGS_benchmark_write_rate_limit = static_cast<uint64_t>(SineRate(0));
4044     }
4045 
4046     if (FLAGS_rate_limiter_bytes_per_sec > 0) {
4047       if (FLAGS_rate_limit_bg_reads &&
4048           !FLAGS_new_table_reader_for_compaction_inputs) {
4049         fprintf(stderr,
4050                 "rate limit compaction reads must have "
4051                 "new_table_reader_for_compaction_inputs set\n");
4052         exit(1);
4053       }
4054       options.rate_limiter.reset(NewGenericRateLimiter(
4055           FLAGS_rate_limiter_bytes_per_sec, 100 * 1000 /* refill_period_us */,
4056           10 /* fairness */,
4057           FLAGS_rate_limit_bg_reads ? RateLimiter::Mode::kReadsOnly
4058                                     : RateLimiter::Mode::kWritesOnly,
4059           FLAGS_rate_limiter_auto_tuned));
4060     }
4061 
4062     options.listeners.emplace_back(listener_);
4063     if (FLAGS_num_multi_db <= 1) {
4064       OpenDb(options, FLAGS_db, &db_);
4065     } else {
4066       multi_dbs_.clear();
4067       multi_dbs_.resize(FLAGS_num_multi_db);
4068       auto wal_dir = options.wal_dir;
4069       for (int i = 0; i < FLAGS_num_multi_db; i++) {
4070         if (!wal_dir.empty()) {
4071           options.wal_dir = GetPathForMultiple(wal_dir, i);
4072         }
4073         OpenDb(options, GetPathForMultiple(FLAGS_db, i), &multi_dbs_[i]);
4074       }
4075       options.wal_dir = wal_dir;
4076     }
4077 
4078     // KeepFilter is a noop filter, this can be used to test compaction filter
4079     if (FLAGS_use_keep_filter) {
4080       options.compaction_filter = new KeepFilter();
4081       fprintf(stdout, "A noop compaction filter is used\n");
4082     }
4083 
4084     if (FLAGS_use_existing_keys) {
4085       // Only work on single database
4086       assert(db_.db != nullptr);
4087       ReadOptions read_opts;
4088       read_opts.total_order_seek = true;
4089       Iterator* iter = db_.db->NewIterator(read_opts);
4090       for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
4091         keys_.emplace_back(iter->key().ToString());
4092       }
4093       delete iter;
4094       FLAGS_num = keys_.size();
4095     }
4096   }
4097 
Open(Options * opts)4098   void Open(Options* opts) {
4099     if (!InitializeOptionsFromFile(opts)) {
4100       InitializeOptionsFromFlags(opts);
4101     }
4102 
4103     InitializeOptionsGeneral(opts);
4104   }
4105 
OpenDb(Options options,const std::string & db_name,DBWithColumnFamilies * db)4106   void OpenDb(Options options, const std::string& db_name,
4107       DBWithColumnFamilies* db) {
4108     Status s;
4109     // Open with column families if necessary.
4110     if (FLAGS_num_column_families > 1) {
4111       size_t num_hot = FLAGS_num_column_families;
4112       if (FLAGS_num_hot_column_families > 0 &&
4113           FLAGS_num_hot_column_families < FLAGS_num_column_families) {
4114         num_hot = FLAGS_num_hot_column_families;
4115       } else {
4116         FLAGS_num_hot_column_families = FLAGS_num_column_families;
4117       }
4118       std::vector<ColumnFamilyDescriptor> column_families;
4119       for (size_t i = 0; i < num_hot; i++) {
4120         column_families.push_back(ColumnFamilyDescriptor(
4121               ColumnFamilyName(i), ColumnFamilyOptions(options)));
4122       }
4123       std::vector<int> cfh_idx_to_prob;
4124       if (!FLAGS_column_family_distribution.empty()) {
4125         std::stringstream cf_prob_stream(FLAGS_column_family_distribution);
4126         std::string cf_prob;
4127         int sum = 0;
4128         while (std::getline(cf_prob_stream, cf_prob, ',')) {
4129           cfh_idx_to_prob.push_back(std::stoi(cf_prob));
4130           sum += cfh_idx_to_prob.back();
4131         }
4132         if (sum != 100) {
4133           fprintf(stderr, "column_family_distribution items must sum to 100\n");
4134           exit(1);
4135         }
4136         if (cfh_idx_to_prob.size() != num_hot) {
4137           fprintf(stderr,
4138                   "got %" ROCKSDB_PRIszt
4139                   " column_family_distribution items; expected "
4140                   "%" ROCKSDB_PRIszt "\n",
4141                   cfh_idx_to_prob.size(), num_hot);
4142           exit(1);
4143         }
4144       }
4145 #ifndef ROCKSDB_LITE
4146       if (FLAGS_readonly) {
4147         s = DB::OpenForReadOnly(options, db_name, column_families,
4148             &db->cfh, &db->db);
4149       } else if (FLAGS_optimistic_transaction_db) {
4150         s = OptimisticTransactionDB::Open(options, db_name, column_families,
4151                                           &db->cfh, &db->opt_txn_db);
4152         if (s.ok()) {
4153           db->db = db->opt_txn_db->GetBaseDB();
4154         }
4155       } else if (FLAGS_transaction_db) {
4156         TransactionDB* ptr;
4157         TransactionDBOptions txn_db_options;
4158         if (options.unordered_write) {
4159           options.two_write_queues = true;
4160           txn_db_options.skip_concurrency_control = true;
4161           txn_db_options.write_policy = WRITE_PREPARED;
4162         }
4163         s = TransactionDB::Open(options, txn_db_options, db_name,
4164                                 column_families, &db->cfh, &ptr);
4165         if (s.ok()) {
4166           db->db = ptr;
4167         }
4168       } else {
4169         s = DB::Open(options, db_name, column_families, &db->cfh, &db->db);
4170       }
4171 #else
4172       s = DB::Open(options, db_name, column_families, &db->cfh, &db->db);
4173 #endif  // ROCKSDB_LITE
4174       db->cfh.resize(FLAGS_num_column_families);
4175       db->num_created = num_hot;
4176       db->num_hot = num_hot;
4177       db->cfh_idx_to_prob = std::move(cfh_idx_to_prob);
4178 #ifndef ROCKSDB_LITE
4179     } else if (FLAGS_readonly) {
4180       s = DB::OpenForReadOnly(options, db_name, &db->db);
4181     } else if (FLAGS_optimistic_transaction_db) {
4182       s = OptimisticTransactionDB::Open(options, db_name, &db->opt_txn_db);
4183       if (s.ok()) {
4184         db->db = db->opt_txn_db->GetBaseDB();
4185       }
4186     } else if (FLAGS_transaction_db) {
4187       TransactionDB* ptr = nullptr;
4188       TransactionDBOptions txn_db_options;
4189       if (options.unordered_write) {
4190         options.two_write_queues = true;
4191         txn_db_options.skip_concurrency_control = true;
4192         txn_db_options.write_policy = WRITE_PREPARED;
4193       }
4194       s = CreateLoggerFromOptions(db_name, options, &options.info_log);
4195       if (s.ok()) {
4196         s = TransactionDB::Open(options, txn_db_options, db_name, &ptr);
4197       }
4198       if (s.ok()) {
4199         db->db = ptr;
4200       }
4201     } else if (FLAGS_use_blob_db) {
4202       blob_db::BlobDBOptions blob_db_options;
4203       blob_db_options.enable_garbage_collection = FLAGS_blob_db_enable_gc;
4204       blob_db_options.garbage_collection_cutoff = FLAGS_blob_db_gc_cutoff;
4205       blob_db_options.is_fifo = FLAGS_blob_db_is_fifo;
4206       blob_db_options.max_db_size = FLAGS_blob_db_max_db_size;
4207       blob_db_options.ttl_range_secs = FLAGS_blob_db_ttl_range_secs;
4208       blob_db_options.min_blob_size = FLAGS_blob_db_min_blob_size;
4209       blob_db_options.bytes_per_sync = FLAGS_blob_db_bytes_per_sync;
4210       blob_db_options.blob_file_size = FLAGS_blob_db_file_size;
4211       blob_db_options.compression = FLAGS_blob_db_compression_type_e;
4212       blob_db::BlobDB* ptr = nullptr;
4213       s = blob_db::BlobDB::Open(options, blob_db_options, db_name, &ptr);
4214       if (s.ok()) {
4215         db->db = ptr;
4216       }
4217     } else if (FLAGS_use_secondary_db) {
4218       if (FLAGS_secondary_path.empty()) {
4219         std::string default_secondary_path;
4220         FLAGS_env->GetTestDirectory(&default_secondary_path);
4221         default_secondary_path += "/dbbench_secondary";
4222         FLAGS_secondary_path = default_secondary_path;
4223       }
4224       s = DB::OpenAsSecondary(options, db_name, FLAGS_secondary_path, &db->db);
4225       if (s.ok() && FLAGS_secondary_update_interval > 0) {
4226         secondary_update_thread_.reset(new port::Thread(
4227             [this](int interval, DBWithColumnFamilies* _db) {
4228               while (0 == secondary_update_stopped_.load(
4229                               std::memory_order_relaxed)) {
4230                 Status secondary_update_status =
4231                     _db->db->TryCatchUpWithPrimary();
4232                 if (!secondary_update_status.ok()) {
4233                   fprintf(stderr, "Failed to catch up with primary: %s\n",
4234                           secondary_update_status.ToString().c_str());
4235                   break;
4236                 }
4237                 ++secondary_db_updates_;
4238                 FLAGS_env->SleepForMicroseconds(interval * 1000000);
4239               }
4240             },
4241             FLAGS_secondary_update_interval, db));
4242       }
4243 #endif  // ROCKSDB_LITE
4244     } else {
4245       s = DB::Open(options, db_name, &db->db);
4246     }
4247     if (!s.ok()) {
4248       fprintf(stderr, "open error: %s\n", s.ToString().c_str());
4249       exit(1);
4250     }
4251   }
4252 
4253   enum WriteMode {
4254     RANDOM, SEQUENTIAL, UNIQUE_RANDOM
4255   };
4256 
WriteSeqDeterministic(ThreadState * thread)4257   void WriteSeqDeterministic(ThreadState* thread) {
4258     DoDeterministicCompact(thread, open_options_.compaction_style, SEQUENTIAL);
4259   }
4260 
WriteUniqueRandomDeterministic(ThreadState * thread)4261   void WriteUniqueRandomDeterministic(ThreadState* thread) {
4262     DoDeterministicCompact(thread, open_options_.compaction_style,
4263                            UNIQUE_RANDOM);
4264   }
4265 
WriteSeq(ThreadState * thread)4266   void WriteSeq(ThreadState* thread) {
4267     DoWrite(thread, SEQUENTIAL);
4268   }
4269 
WriteRandom(ThreadState * thread)4270   void WriteRandom(ThreadState* thread) {
4271     DoWrite(thread, RANDOM);
4272   }
4273 
WriteUniqueRandom(ThreadState * thread)4274   void WriteUniqueRandom(ThreadState* thread) {
4275     DoWrite(thread, UNIQUE_RANDOM);
4276   }
4277 
4278   class KeyGenerator {
4279    public:
KeyGenerator(Random64 * rand,WriteMode mode,uint64_t num,uint64_t=64* 1024)4280     KeyGenerator(Random64* rand, WriteMode mode, uint64_t num,
4281                  uint64_t /*num_per_set*/ = 64 * 1024)
4282         : rand_(rand), mode_(mode), num_(num), next_(0) {
4283       if (mode_ == UNIQUE_RANDOM) {
4284         // NOTE: if memory consumption of this approach becomes a concern,
4285         // we can either break it into pieces and only random shuffle a section
4286         // each time. Alternatively, use a bit map implementation
4287         // (https://reviews.facebook.net/differential/diff/54627/)
4288         values_.resize(num_);
4289         for (uint64_t i = 0; i < num_; ++i) {
4290           values_[i] = i;
4291         }
4292         std::shuffle(
4293             values_.begin(), values_.end(),
4294             std::default_random_engine(static_cast<unsigned int>(FLAGS_seed)));
4295       }
4296     }
4297 
Next()4298     uint64_t Next() {
4299       switch (mode_) {
4300         case SEQUENTIAL:
4301           return next_++;
4302         case RANDOM:
4303           return rand_->Next() % num_;
4304         case UNIQUE_RANDOM:
4305           assert(next_ < num_);
4306           return values_[next_++];
4307       }
4308       assert(false);
4309       return std::numeric_limits<uint64_t>::max();
4310     }
4311 
4312    private:
4313     Random64* rand_;
4314     WriteMode mode_;
4315     const uint64_t num_;
4316     uint64_t next_;
4317     std::vector<uint64_t> values_;
4318   };
4319 
SelectDB(ThreadState * thread)4320   DB* SelectDB(ThreadState* thread) {
4321     return SelectDBWithCfh(thread)->db;
4322   }
4323 
SelectDBWithCfh(ThreadState * thread)4324   DBWithColumnFamilies* SelectDBWithCfh(ThreadState* thread) {
4325     return SelectDBWithCfh(thread->rand.Next());
4326   }
4327 
SelectDBWithCfh(uint64_t rand_int)4328   DBWithColumnFamilies* SelectDBWithCfh(uint64_t rand_int) {
4329     if (db_.db != nullptr) {
4330       return &db_;
4331     } else  {
4332       return &multi_dbs_[rand_int % multi_dbs_.size()];
4333     }
4334   }
4335 
SineRate(double x)4336   double SineRate(double x) {
4337     return FLAGS_sine_a*sin((FLAGS_sine_b*x) + FLAGS_sine_c) + FLAGS_sine_d;
4338   }
4339 
DoWrite(ThreadState * thread,WriteMode write_mode)4340   void DoWrite(ThreadState* thread, WriteMode write_mode) {
4341     const int test_duration = write_mode == RANDOM ? FLAGS_duration : 0;
4342     const int64_t num_ops = writes_ == 0 ? num_ : writes_;
4343 
4344     size_t num_key_gens = 1;
4345     if (db_.db == nullptr) {
4346       num_key_gens = multi_dbs_.size();
4347     }
4348     std::vector<std::unique_ptr<KeyGenerator>> key_gens(num_key_gens);
4349     int64_t max_ops = num_ops * num_key_gens;
4350     int64_t ops_per_stage = max_ops;
4351     if (FLAGS_num_column_families > 1 && FLAGS_num_hot_column_families > 0) {
4352       ops_per_stage = (max_ops - 1) / (FLAGS_num_column_families /
4353                                        FLAGS_num_hot_column_families) +
4354                       1;
4355     }
4356 
4357     Duration duration(test_duration, max_ops, ops_per_stage);
4358     for (size_t i = 0; i < num_key_gens; i++) {
4359       key_gens[i].reset(new KeyGenerator(&(thread->rand), write_mode,
4360                                          num_ + max_num_range_tombstones_,
4361                                          ops_per_stage));
4362     }
4363 
4364     if (num_ != FLAGS_num) {
4365       char msg[100];
4366       snprintf(msg, sizeof(msg), "(%" PRIu64 " ops)", num_);
4367       thread->stats.AddMessage(msg);
4368     }
4369 
4370     RandomGenerator gen;
4371     WriteBatch batch;
4372     Status s;
4373     int64_t bytes = 0;
4374 
4375     std::unique_ptr<const char[]> key_guard;
4376     Slice key = AllocateKey(&key_guard);
4377     std::unique_ptr<const char[]> begin_key_guard;
4378     Slice begin_key = AllocateKey(&begin_key_guard);
4379     std::unique_ptr<const char[]> end_key_guard;
4380     Slice end_key = AllocateKey(&end_key_guard);
4381     std::vector<std::unique_ptr<const char[]>> expanded_key_guards;
4382     std::vector<Slice> expanded_keys;
4383     if (FLAGS_expand_range_tombstones) {
4384       expanded_key_guards.resize(range_tombstone_width_);
4385       for (auto& expanded_key_guard : expanded_key_guards) {
4386         expanded_keys.emplace_back(AllocateKey(&expanded_key_guard));
4387       }
4388     }
4389 
4390     int64_t stage = 0;
4391     int64_t num_written = 0;
4392     while (!duration.Done(entries_per_batch_)) {
4393       if (duration.GetStage() != stage) {
4394         stage = duration.GetStage();
4395         if (db_.db != nullptr) {
4396           db_.CreateNewCf(open_options_, stage);
4397         } else {
4398           for (auto& db : multi_dbs_) {
4399             db.CreateNewCf(open_options_, stage);
4400           }
4401         }
4402       }
4403 
4404       size_t id = thread->rand.Next() % num_key_gens;
4405       DBWithColumnFamilies* db_with_cfh = SelectDBWithCfh(id);
4406       batch.Clear();
4407       int64_t batch_bytes = 0;
4408 
4409       for (int64_t j = 0; j < entries_per_batch_; j++) {
4410         int64_t rand_num = key_gens[id]->Next();
4411         GenerateKeyFromInt(rand_num, FLAGS_num, &key);
4412         Slice val = gen.Generate();
4413         if (use_blob_db_) {
4414 #ifndef ROCKSDB_LITE
4415           blob_db::BlobDB* blobdb =
4416               static_cast<blob_db::BlobDB*>(db_with_cfh->db);
4417           if (FLAGS_blob_db_max_ttl_range > 0) {
4418             int ttl = rand() % FLAGS_blob_db_max_ttl_range;
4419             s = blobdb->PutWithTTL(write_options_, key, val, ttl);
4420           } else {
4421             s = blobdb->Put(write_options_, key, val);
4422           }
4423 #endif  //  ROCKSDB_LITE
4424         } else if (FLAGS_num_column_families <= 1) {
4425           batch.Put(key, val);
4426         } else {
4427           // We use same rand_num as seed for key and column family so that we
4428           // can deterministically find the cfh corresponding to a particular
4429           // key while reading the key.
4430           batch.Put(db_with_cfh->GetCfh(rand_num), key,
4431                     val);
4432         }
4433         batch_bytes += val.size() + key_size_;
4434         bytes += val.size() + key_size_;
4435         ++num_written;
4436         if (writes_per_range_tombstone_ > 0 &&
4437             num_written > writes_before_delete_range_ &&
4438             (num_written - writes_before_delete_range_) /
4439                     writes_per_range_tombstone_ <=
4440                 max_num_range_tombstones_ &&
4441             (num_written - writes_before_delete_range_) %
4442                     writes_per_range_tombstone_ ==
4443                 0) {
4444           int64_t begin_num = key_gens[id]->Next();
4445           if (FLAGS_expand_range_tombstones) {
4446             for (int64_t offset = 0; offset < range_tombstone_width_;
4447                  ++offset) {
4448               GenerateKeyFromInt(begin_num + offset, FLAGS_num,
4449                                  &expanded_keys[offset]);
4450               if (use_blob_db_) {
4451 #ifndef ROCKSDB_LITE
4452                 s = db_with_cfh->db->Delete(write_options_,
4453                                             expanded_keys[offset]);
4454 #endif  //  ROCKSDB_LITE
4455               } else if (FLAGS_num_column_families <= 1) {
4456                 batch.Delete(expanded_keys[offset]);
4457               } else {
4458                 batch.Delete(db_with_cfh->GetCfh(rand_num),
4459                              expanded_keys[offset]);
4460               }
4461             }
4462           } else {
4463             GenerateKeyFromInt(begin_num, FLAGS_num, &begin_key);
4464             GenerateKeyFromInt(begin_num + range_tombstone_width_, FLAGS_num,
4465                                &end_key);
4466             if (use_blob_db_) {
4467 #ifndef ROCKSDB_LITE
4468               s = db_with_cfh->db->DeleteRange(
4469                   write_options_, db_with_cfh->db->DefaultColumnFamily(),
4470                   begin_key, end_key);
4471 #endif  //  ROCKSDB_LITE
4472             } else if (FLAGS_num_column_families <= 1) {
4473               batch.DeleteRange(begin_key, end_key);
4474             } else {
4475               batch.DeleteRange(db_with_cfh->GetCfh(rand_num), begin_key,
4476                                 end_key);
4477             }
4478           }
4479         }
4480       }
4481       if (thread->shared->write_rate_limiter.get() != nullptr) {
4482         thread->shared->write_rate_limiter->Request(
4483             batch_bytes, Env::IO_HIGH,
4484             nullptr /* stats */, RateLimiter::OpType::kWrite);
4485         // Set time at which last op finished to Now() to hide latency and
4486         // sleep from rate limiter. Also, do the check once per batch, not
4487         // once per write.
4488         thread->stats.ResetLastOpTime();
4489       }
4490       if (!use_blob_db_) {
4491         s = db_with_cfh->db->Write(write_options_, &batch);
4492       }
4493       thread->stats.FinishedOps(db_with_cfh, db_with_cfh->db,
4494                                 entries_per_batch_, kWrite);
4495       if (FLAGS_sine_write_rate) {
4496         uint64_t now = FLAGS_env->NowMicros();
4497 
4498         uint64_t usecs_since_last;
4499         if (now > thread->stats.GetSineInterval()) {
4500           usecs_since_last = now - thread->stats.GetSineInterval();
4501         } else {
4502           usecs_since_last = 0;
4503         }
4504 
4505         if (usecs_since_last >
4506             (FLAGS_sine_write_rate_interval_milliseconds * uint64_t{1000})) {
4507           double usecs_since_start =
4508                   static_cast<double>(now - thread->stats.GetStart());
4509           thread->stats.ResetSineInterval();
4510           uint64_t write_rate =
4511                   static_cast<uint64_t>(SineRate(usecs_since_start / 1000000.0));
4512           thread->shared->write_rate_limiter.reset(
4513                   NewGenericRateLimiter(write_rate));
4514         }
4515       }
4516       if (!s.ok()) {
4517         s = listener_->WaitForRecovery(600000000) ? Status::OK() : s;
4518       }
4519 
4520       if (!s.ok()) {
4521         fprintf(stderr, "put error: %s\n", s.ToString().c_str());
4522         exit(1);
4523       }
4524     }
4525     thread->stats.AddBytes(bytes);
4526   }
4527 
DoDeterministicCompact(ThreadState * thread,CompactionStyle compaction_style,WriteMode write_mode)4528   Status DoDeterministicCompact(ThreadState* thread,
4529                                 CompactionStyle compaction_style,
4530                                 WriteMode write_mode) {
4531 #ifndef ROCKSDB_LITE
4532     ColumnFamilyMetaData meta;
4533     std::vector<DB*> db_list;
4534     if (db_.db != nullptr) {
4535       db_list.push_back(db_.db);
4536     } else {
4537       for (auto& db : multi_dbs_) {
4538         db_list.push_back(db.db);
4539       }
4540     }
4541     std::vector<Options> options_list;
4542     for (auto db : db_list) {
4543       options_list.push_back(db->GetOptions());
4544       if (compaction_style != kCompactionStyleFIFO) {
4545         db->SetOptions({{"disable_auto_compactions", "1"},
4546                         {"level0_slowdown_writes_trigger", "400000000"},
4547                         {"level0_stop_writes_trigger", "400000000"}});
4548       } else {
4549         db->SetOptions({{"disable_auto_compactions", "1"}});
4550       }
4551     }
4552 
4553     assert(!db_list.empty());
4554     auto num_db = db_list.size();
4555     size_t num_levels = static_cast<size_t>(open_options_.num_levels);
4556     size_t output_level = open_options_.num_levels - 1;
4557     std::vector<std::vector<std::vector<SstFileMetaData>>> sorted_runs(num_db);
4558     std::vector<size_t> num_files_at_level0(num_db, 0);
4559     if (compaction_style == kCompactionStyleLevel) {
4560       if (num_levels == 0) {
4561         return Status::InvalidArgument("num_levels should be larger than 1");
4562       }
4563       bool should_stop = false;
4564       while (!should_stop) {
4565         if (sorted_runs[0].empty()) {
4566           DoWrite(thread, write_mode);
4567         } else {
4568           DoWrite(thread, UNIQUE_RANDOM);
4569         }
4570         for (size_t i = 0; i < num_db; i++) {
4571           auto db = db_list[i];
4572           db->Flush(FlushOptions());
4573           db->GetColumnFamilyMetaData(&meta);
4574           if (num_files_at_level0[i] == meta.levels[0].files.size() ||
4575               writes_ == 0) {
4576             should_stop = true;
4577             continue;
4578           }
4579           sorted_runs[i].emplace_back(
4580               meta.levels[0].files.begin(),
4581               meta.levels[0].files.end() - num_files_at_level0[i]);
4582           num_files_at_level0[i] = meta.levels[0].files.size();
4583           if (sorted_runs[i].back().size() == 1) {
4584             should_stop = true;
4585             continue;
4586           }
4587           if (sorted_runs[i].size() == output_level) {
4588             auto& L1 = sorted_runs[i].back();
4589             L1.erase(L1.begin(), L1.begin() + L1.size() / 3);
4590             should_stop = true;
4591             continue;
4592           }
4593         }
4594         writes_ /= static_cast<int64_t>(open_options_.max_bytes_for_level_multiplier);
4595       }
4596       for (size_t i = 0; i < num_db; i++) {
4597         if (sorted_runs[i].size() < num_levels - 1) {
4598           fprintf(stderr, "n is too small to fill %" ROCKSDB_PRIszt " levels\n", num_levels);
4599           exit(1);
4600         }
4601       }
4602       for (size_t i = 0; i < num_db; i++) {
4603         auto db = db_list[i];
4604         auto compactionOptions = CompactionOptions();
4605         compactionOptions.compression = FLAGS_compression_type_e;
4606         auto options = db->GetOptions();
4607         MutableCFOptions mutable_cf_options(options);
4608         for (size_t j = 0; j < sorted_runs[i].size(); j++) {
4609           compactionOptions.output_file_size_limit =
4610               MaxFileSizeForLevel(mutable_cf_options,
4611                   static_cast<int>(output_level), compaction_style);
4612           std::cout << sorted_runs[i][j].size() << std::endl;
4613           db->CompactFiles(compactionOptions, {sorted_runs[i][j].back().name,
4614                                                sorted_runs[i][j].front().name},
4615                            static_cast<int>(output_level - j) /*level*/);
4616         }
4617       }
4618     } else if (compaction_style == kCompactionStyleUniversal) {
4619       auto ratio = open_options_.compaction_options_universal.size_ratio;
4620       bool should_stop = false;
4621       while (!should_stop) {
4622         if (sorted_runs[0].empty()) {
4623           DoWrite(thread, write_mode);
4624         } else {
4625           DoWrite(thread, UNIQUE_RANDOM);
4626         }
4627         for (size_t i = 0; i < num_db; i++) {
4628           auto db = db_list[i];
4629           db->Flush(FlushOptions());
4630           db->GetColumnFamilyMetaData(&meta);
4631           if (num_files_at_level0[i] == meta.levels[0].files.size() ||
4632               writes_ == 0) {
4633             should_stop = true;
4634             continue;
4635           }
4636           sorted_runs[i].emplace_back(
4637               meta.levels[0].files.begin(),
4638               meta.levels[0].files.end() - num_files_at_level0[i]);
4639           num_files_at_level0[i] = meta.levels[0].files.size();
4640           if (sorted_runs[i].back().size() == 1) {
4641             should_stop = true;
4642             continue;
4643           }
4644           num_files_at_level0[i] = meta.levels[0].files.size();
4645         }
4646         writes_ =  static_cast<int64_t>(writes_* static_cast<double>(100) / (ratio + 200));
4647       }
4648       for (size_t i = 0; i < num_db; i++) {
4649         if (sorted_runs[i].size() < num_levels) {
4650           fprintf(stderr, "n is too small to fill %" ROCKSDB_PRIszt  " levels\n", num_levels);
4651           exit(1);
4652         }
4653       }
4654       for (size_t i = 0; i < num_db; i++) {
4655         auto db = db_list[i];
4656         auto compactionOptions = CompactionOptions();
4657         compactionOptions.compression = FLAGS_compression_type_e;
4658         auto options = db->GetOptions();
4659         MutableCFOptions mutable_cf_options(options);
4660         for (size_t j = 0; j < sorted_runs[i].size(); j++) {
4661           compactionOptions.output_file_size_limit =
4662               MaxFileSizeForLevel(mutable_cf_options,
4663                   static_cast<int>(output_level), compaction_style);
4664           db->CompactFiles(
4665               compactionOptions,
4666               {sorted_runs[i][j].back().name, sorted_runs[i][j].front().name},
4667               (output_level > j ? static_cast<int>(output_level - j)
4668                                 : 0) /*level*/);
4669         }
4670       }
4671     } else if (compaction_style == kCompactionStyleFIFO) {
4672       if (num_levels != 1) {
4673         return Status::InvalidArgument(
4674           "num_levels should be 1 for FIFO compaction");
4675       }
4676       if (FLAGS_num_multi_db != 0) {
4677         return Status::InvalidArgument("Doesn't support multiDB");
4678       }
4679       auto db = db_list[0];
4680       std::vector<std::string> file_names;
4681       while (true) {
4682         if (sorted_runs[0].empty()) {
4683           DoWrite(thread, write_mode);
4684         } else {
4685           DoWrite(thread, UNIQUE_RANDOM);
4686         }
4687         db->Flush(FlushOptions());
4688         db->GetColumnFamilyMetaData(&meta);
4689         auto total_size = meta.levels[0].size;
4690         if (total_size >=
4691           db->GetOptions().compaction_options_fifo.max_table_files_size) {
4692           for (auto file_meta : meta.levels[0].files) {
4693             file_names.emplace_back(file_meta.name);
4694           }
4695           break;
4696         }
4697       }
4698       // TODO(shuzhang1989): Investigate why CompactFiles not working
4699       // auto compactionOptions = CompactionOptions();
4700       // db->CompactFiles(compactionOptions, file_names, 0);
4701       auto compactionOptions = CompactRangeOptions();
4702       db->CompactRange(compactionOptions, nullptr, nullptr);
4703     } else {
4704       fprintf(stdout,
4705               "%-12s : skipped (-compaction_stype=kCompactionStyleNone)\n",
4706               "filldeterministic");
4707       return Status::InvalidArgument("None compaction is not supported");
4708     }
4709 
4710 // Verify seqno and key range
4711 // Note: the seqno get changed at the max level by implementation
4712 // optimization, so skip the check of the max level.
4713 #ifndef NDEBUG
4714     for (size_t k = 0; k < num_db; k++) {
4715       auto db = db_list[k];
4716       db->GetColumnFamilyMetaData(&meta);
4717       // verify the number of sorted runs
4718       if (compaction_style == kCompactionStyleLevel) {
4719         assert(num_levels - 1 == sorted_runs[k].size());
4720       } else if (compaction_style == kCompactionStyleUniversal) {
4721         assert(meta.levels[0].files.size() + num_levels - 1 ==
4722                sorted_runs[k].size());
4723       } else if (compaction_style == kCompactionStyleFIFO) {
4724         // TODO(gzh): FIFO compaction
4725         db->GetColumnFamilyMetaData(&meta);
4726         auto total_size = meta.levels[0].size;
4727         assert(total_size <=
4728           db->GetOptions().compaction_options_fifo.max_table_files_size);
4729           break;
4730       }
4731 
4732       // verify smallest/largest seqno and key range of each sorted run
4733       auto max_level = num_levels - 1;
4734       int level;
4735       for (size_t i = 0; i < sorted_runs[k].size(); i++) {
4736         level = static_cast<int>(max_level - i);
4737         SequenceNumber sorted_run_smallest_seqno = kMaxSequenceNumber;
4738         SequenceNumber sorted_run_largest_seqno = 0;
4739         std::string sorted_run_smallest_key, sorted_run_largest_key;
4740         bool first_key = true;
4741         for (auto fileMeta : sorted_runs[k][i]) {
4742           sorted_run_smallest_seqno =
4743               std::min(sorted_run_smallest_seqno, fileMeta.smallest_seqno);
4744           sorted_run_largest_seqno =
4745               std::max(sorted_run_largest_seqno, fileMeta.largest_seqno);
4746           if (first_key ||
4747               db->DefaultColumnFamily()->GetComparator()->Compare(
4748                   fileMeta.smallestkey, sorted_run_smallest_key) < 0) {
4749             sorted_run_smallest_key = fileMeta.smallestkey;
4750           }
4751           if (first_key ||
4752               db->DefaultColumnFamily()->GetComparator()->Compare(
4753                   fileMeta.largestkey, sorted_run_largest_key) > 0) {
4754             sorted_run_largest_key = fileMeta.largestkey;
4755           }
4756           first_key = false;
4757         }
4758         if (compaction_style == kCompactionStyleLevel ||
4759             (compaction_style == kCompactionStyleUniversal && level > 0)) {
4760           SequenceNumber level_smallest_seqno = kMaxSequenceNumber;
4761           SequenceNumber level_largest_seqno = 0;
4762           for (auto fileMeta : meta.levels[level].files) {
4763             level_smallest_seqno =
4764                 std::min(level_smallest_seqno, fileMeta.smallest_seqno);
4765             level_largest_seqno =
4766                 std::max(level_largest_seqno, fileMeta.largest_seqno);
4767           }
4768           assert(sorted_run_smallest_key ==
4769                  meta.levels[level].files.front().smallestkey);
4770           assert(sorted_run_largest_key ==
4771                  meta.levels[level].files.back().largestkey);
4772           if (level != static_cast<int>(max_level)) {
4773             // compaction at max_level would change sequence number
4774             assert(sorted_run_smallest_seqno == level_smallest_seqno);
4775             assert(sorted_run_largest_seqno == level_largest_seqno);
4776           }
4777         } else if (compaction_style == kCompactionStyleUniversal) {
4778           // level <= 0 means sorted runs on level 0
4779           auto level0_file =
4780               meta.levels[0].files[sorted_runs[k].size() - 1 - i];
4781           assert(sorted_run_smallest_key == level0_file.smallestkey);
4782           assert(sorted_run_largest_key == level0_file.largestkey);
4783           if (level != static_cast<int>(max_level)) {
4784             assert(sorted_run_smallest_seqno == level0_file.smallest_seqno);
4785             assert(sorted_run_largest_seqno == level0_file.largest_seqno);
4786           }
4787         }
4788       }
4789     }
4790 #endif
4791     // print the size of each sorted_run
4792     for (size_t k = 0; k < num_db; k++) {
4793       auto db = db_list[k];
4794       fprintf(stdout,
4795               "---------------------- DB %" ROCKSDB_PRIszt " LSM ---------------------\n", k);
4796       db->GetColumnFamilyMetaData(&meta);
4797       for (auto& levelMeta : meta.levels) {
4798         if (levelMeta.files.empty()) {
4799           continue;
4800         }
4801         if (levelMeta.level == 0) {
4802           for (auto& fileMeta : levelMeta.files) {
4803             fprintf(stdout, "Level[%d]: %s(size: %" ROCKSDB_PRIszt " bytes)\n",
4804                     levelMeta.level, fileMeta.name.c_str(), fileMeta.size);
4805           }
4806         } else {
4807           fprintf(stdout, "Level[%d]: %s - %s(total size: %" PRIi64 " bytes)\n",
4808                   levelMeta.level, levelMeta.files.front().name.c_str(),
4809                   levelMeta.files.back().name.c_str(), levelMeta.size);
4810         }
4811       }
4812     }
4813     for (size_t i = 0; i < num_db; i++) {
4814       db_list[i]->SetOptions(
4815           {{"disable_auto_compactions",
4816             std::to_string(options_list[i].disable_auto_compactions)},
4817            {"level0_slowdown_writes_trigger",
4818             std::to_string(options_list[i].level0_slowdown_writes_trigger)},
4819            {"level0_stop_writes_trigger",
4820             std::to_string(options_list[i].level0_stop_writes_trigger)}});
4821     }
4822     return Status::OK();
4823 #else
4824     (void)thread;
4825     (void)compaction_style;
4826     (void)write_mode;
4827     fprintf(stderr, "Rocksdb Lite doesn't support filldeterministic\n");
4828     return Status::NotSupported(
4829         "Rocksdb Lite doesn't support filldeterministic");
4830 #endif  // ROCKSDB_LITE
4831   }
4832 
ReadSequential(ThreadState * thread)4833   void ReadSequential(ThreadState* thread) {
4834     if (db_.db != nullptr) {
4835       ReadSequential(thread, db_.db);
4836     } else {
4837       for (const auto& db_with_cfh : multi_dbs_) {
4838         ReadSequential(thread, db_with_cfh.db);
4839       }
4840     }
4841   }
4842 
ReadSequential(ThreadState * thread,DB * db)4843   void ReadSequential(ThreadState* thread, DB* db) {
4844     ReadOptions options(FLAGS_verify_checksum, true);
4845     options.tailing = FLAGS_use_tailing_iterator;
4846 
4847     Iterator* iter = db->NewIterator(options);
4848     int64_t i = 0;
4849     int64_t bytes = 0;
4850     for (iter->SeekToFirst(); i < reads_ && iter->Valid(); iter->Next()) {
4851       bytes += iter->key().size() + iter->value().size();
4852       thread->stats.FinishedOps(nullptr, db, 1, kRead);
4853       ++i;
4854 
4855       if (thread->shared->read_rate_limiter.get() != nullptr &&
4856           i % 1024 == 1023) {
4857         thread->shared->read_rate_limiter->Request(1024, Env::IO_HIGH,
4858                                                    nullptr /* stats */,
4859                                                    RateLimiter::OpType::kRead);
4860       }
4861     }
4862 
4863     delete iter;
4864     thread->stats.AddBytes(bytes);
4865     if (FLAGS_perf_level > ROCKSDB_NAMESPACE::PerfLevel::kDisable) {
4866       thread->stats.AddMessage(std::string("PERF_CONTEXT:\n") +
4867                                get_perf_context()->ToString());
4868     }
4869   }
4870 
ReadToRowCache(ThreadState * thread)4871   void ReadToRowCache(ThreadState* thread) {
4872     int64_t read = 0;
4873     int64_t found = 0;
4874     int64_t bytes = 0;
4875     int64_t key_rand = 0;
4876     ReadOptions options(FLAGS_verify_checksum, true);
4877     std::unique_ptr<const char[]> key_guard;
4878     Slice key = AllocateKey(&key_guard);
4879     PinnableSlice pinnable_val;
4880 
4881     while (key_rand < FLAGS_num) {
4882       DBWithColumnFamilies* db_with_cfh = SelectDBWithCfh(thread);
4883       // We use same key_rand as seed for key and column family so that we can
4884       // deterministically find the cfh corresponding to a particular key, as it
4885       // is done in DoWrite method.
4886       GenerateKeyFromInt(key_rand, FLAGS_num, &key);
4887       key_rand++;
4888       read++;
4889       Status s;
4890       if (FLAGS_num_column_families > 1) {
4891         s = db_with_cfh->db->Get(options, db_with_cfh->GetCfh(key_rand), key,
4892                                  &pinnable_val);
4893       } else {
4894         pinnable_val.Reset();
4895         s = db_with_cfh->db->Get(options,
4896                                  db_with_cfh->db->DefaultColumnFamily(), key,
4897                                  &pinnable_val);
4898       }
4899 
4900       if (s.ok()) {
4901         found++;
4902         bytes += key.size() + pinnable_val.size();
4903       } else if (!s.IsNotFound()) {
4904         fprintf(stderr, "Get returned an error: %s\n", s.ToString().c_str());
4905         abort();
4906       }
4907 
4908       if (thread->shared->read_rate_limiter.get() != nullptr &&
4909           read % 256 == 255) {
4910         thread->shared->read_rate_limiter->Request(
4911             256, Env::IO_HIGH, nullptr /* stats */, RateLimiter::OpType::kRead);
4912       }
4913 
4914       thread->stats.FinishedOps(db_with_cfh, db_with_cfh->db, 1, kRead);
4915     }
4916 
4917     char msg[100];
4918     snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found)\n", found,
4919              read);
4920 
4921     thread->stats.AddBytes(bytes);
4922     thread->stats.AddMessage(msg);
4923 
4924     if (FLAGS_perf_level > ROCKSDB_NAMESPACE::PerfLevel::kDisable) {
4925       thread->stats.AddMessage(std::string("PERF_CONTEXT:\n") +
4926                                get_perf_context()->ToString());
4927     }
4928   }
4929 
ReadReverse(ThreadState * thread)4930   void ReadReverse(ThreadState* thread) {
4931     if (db_.db != nullptr) {
4932       ReadReverse(thread, db_.db);
4933     } else {
4934       for (const auto& db_with_cfh : multi_dbs_) {
4935         ReadReverse(thread, db_with_cfh.db);
4936       }
4937     }
4938   }
4939 
ReadReverse(ThreadState * thread,DB * db)4940   void ReadReverse(ThreadState* thread, DB* db) {
4941     Iterator* iter = db->NewIterator(ReadOptions(FLAGS_verify_checksum, true));
4942     int64_t i = 0;
4943     int64_t bytes = 0;
4944     for (iter->SeekToLast(); i < reads_ && iter->Valid(); iter->Prev()) {
4945       bytes += iter->key().size() + iter->value().size();
4946       thread->stats.FinishedOps(nullptr, db, 1, kRead);
4947       ++i;
4948       if (thread->shared->read_rate_limiter.get() != nullptr &&
4949           i % 1024 == 1023) {
4950         thread->shared->read_rate_limiter->Request(1024, Env::IO_HIGH,
4951                                                    nullptr /* stats */,
4952                                                    RateLimiter::OpType::kRead);
4953       }
4954     }
4955     delete iter;
4956     thread->stats.AddBytes(bytes);
4957   }
4958 
ReadRandomFast(ThreadState * thread)4959   void ReadRandomFast(ThreadState* thread) {
4960     int64_t read = 0;
4961     int64_t found = 0;
4962     int64_t nonexist = 0;
4963     ReadOptions options(FLAGS_verify_checksum, true);
4964     std::unique_ptr<const char[]> key_guard;
4965     Slice key = AllocateKey(&key_guard);
4966     std::string value;
4967     DB* db = SelectDBWithCfh(thread)->db;
4968 
4969     int64_t pot = 1;
4970     while (pot < FLAGS_num) {
4971       pot <<= 1;
4972     }
4973 
4974     Duration duration(FLAGS_duration, reads_);
4975     do {
4976       for (int i = 0; i < 100; ++i) {
4977         int64_t key_rand = thread->rand.Next() & (pot - 1);
4978         GenerateKeyFromInt(key_rand, FLAGS_num, &key);
4979         ++read;
4980         auto status = db->Get(options, key, &value);
4981         if (status.ok()) {
4982           ++found;
4983         } else if (!status.IsNotFound()) {
4984           fprintf(stderr, "Get returned an error: %s\n",
4985                   status.ToString().c_str());
4986           abort();
4987         }
4988         if (key_rand >= FLAGS_num) {
4989           ++nonexist;
4990         }
4991       }
4992       if (thread->shared->read_rate_limiter.get() != nullptr) {
4993         thread->shared->read_rate_limiter->Request(
4994             100, Env::IO_HIGH, nullptr /* stats */, RateLimiter::OpType::kRead);
4995       }
4996 
4997       thread->stats.FinishedOps(nullptr, db, 100, kRead);
4998     } while (!duration.Done(100));
4999 
5000     char msg[100];
5001     snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found, "
5002              "issued %" PRIu64 " non-exist keys)\n",
5003              found, read, nonexist);
5004 
5005     thread->stats.AddMessage(msg);
5006 
5007     if (FLAGS_perf_level > ROCKSDB_NAMESPACE::PerfLevel::kDisable) {
5008       thread->stats.AddMessage(std::string("PERF_CONTEXT:\n") +
5009                                get_perf_context()->ToString());
5010     }
5011   }
5012 
GetRandomKey(Random64 * rand)5013   int64_t GetRandomKey(Random64* rand) {
5014     uint64_t rand_int = rand->Next();
5015     int64_t key_rand;
5016     if (read_random_exp_range_ == 0) {
5017       key_rand = rand_int % FLAGS_num;
5018     } else {
5019       const uint64_t kBigInt = static_cast<uint64_t>(1U) << 62;
5020       long double order = -static_cast<long double>(rand_int % kBigInt) /
5021                           static_cast<long double>(kBigInt) *
5022                           read_random_exp_range_;
5023       long double exp_ran = std::exp(order);
5024       uint64_t rand_num =
5025           static_cast<int64_t>(exp_ran * static_cast<long double>(FLAGS_num));
5026       // Map to a different number to avoid locality.
5027       const uint64_t kBigPrime = 0x5bd1e995;
5028       // Overflow is like %(2^64). Will have little impact of results.
5029       key_rand = static_cast<int64_t>((rand_num * kBigPrime) % FLAGS_num);
5030     }
5031     return key_rand;
5032   }
5033 
ReadRandom(ThreadState * thread)5034   void ReadRandom(ThreadState* thread) {
5035     int64_t read = 0;
5036     int64_t found = 0;
5037     int64_t bytes = 0;
5038     int num_keys = 0;
5039     int64_t key_rand = GetRandomKey(&thread->rand);
5040     ReadOptions options(FLAGS_verify_checksum, true);
5041     std::unique_ptr<const char[]> key_guard;
5042     Slice key = AllocateKey(&key_guard);
5043     PinnableSlice pinnable_val;
5044 
5045     Duration duration(FLAGS_duration, reads_);
5046     while (!duration.Done(1)) {
5047       DBWithColumnFamilies* db_with_cfh = SelectDBWithCfh(thread);
5048       // We use same key_rand as seed for key and column family so that we can
5049       // deterministically find the cfh corresponding to a particular key, as it
5050       // is done in DoWrite method.
5051       GenerateKeyFromInt(key_rand, FLAGS_num, &key);
5052       if (entries_per_batch_ > 1 && FLAGS_multiread_stride) {
5053         if (++num_keys == entries_per_batch_) {
5054           num_keys = 0;
5055           key_rand = GetRandomKey(&thread->rand);
5056           if ((key_rand + (entries_per_batch_ - 1) * FLAGS_multiread_stride) >=
5057               FLAGS_num) {
5058             key_rand = FLAGS_num - entries_per_batch_ * FLAGS_multiread_stride;
5059           }
5060         } else {
5061           key_rand += FLAGS_multiread_stride;
5062         }
5063       } else {
5064         key_rand = GetRandomKey(&thread->rand);
5065       }
5066       read++;
5067       Status s;
5068       if (FLAGS_num_column_families > 1) {
5069         s = db_with_cfh->db->Get(options, db_with_cfh->GetCfh(key_rand), key,
5070                                  &pinnable_val);
5071       } else {
5072         pinnable_val.Reset();
5073         s = db_with_cfh->db->Get(options,
5074                                  db_with_cfh->db->DefaultColumnFamily(), key,
5075                                  &pinnable_val);
5076       }
5077       if (s.ok()) {
5078         found++;
5079         bytes += key.size() + pinnable_val.size();
5080       } else if (!s.IsNotFound()) {
5081         fprintf(stderr, "Get returned an error: %s\n", s.ToString().c_str());
5082         abort();
5083       }
5084 
5085       if (thread->shared->read_rate_limiter.get() != nullptr &&
5086           read % 256 == 255) {
5087         thread->shared->read_rate_limiter->Request(
5088             256, Env::IO_HIGH, nullptr /* stats */, RateLimiter::OpType::kRead);
5089       }
5090 
5091       thread->stats.FinishedOps(db_with_cfh, db_with_cfh->db, 1, kRead);
5092     }
5093 
5094     char msg[100];
5095     snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found)\n",
5096              found, read);
5097 
5098     thread->stats.AddBytes(bytes);
5099     thread->stats.AddMessage(msg);
5100 
5101     if (FLAGS_perf_level > ROCKSDB_NAMESPACE::PerfLevel::kDisable) {
5102       thread->stats.AddMessage(std::string("PERF_CONTEXT:\n") +
5103                                get_perf_context()->ToString());
5104     }
5105   }
5106 
5107   // Calls MultiGet over a list of keys from a random distribution.
5108   // Returns the total number of keys found.
MultiReadRandom(ThreadState * thread)5109   void MultiReadRandom(ThreadState* thread) {
5110     int64_t read = 0;
5111     int64_t num_multireads = 0;
5112     int64_t found = 0;
5113     ReadOptions options(FLAGS_verify_checksum, true);
5114     std::vector<Slice> keys;
5115     std::vector<std::unique_ptr<const char[]> > key_guards;
5116     std::vector<std::string> values(entries_per_batch_);
5117     PinnableSlice* pin_values = new PinnableSlice[entries_per_batch_];
5118     std::unique_ptr<PinnableSlice[]> pin_values_guard(pin_values);
5119     std::vector<Status> stat_list(entries_per_batch_);
5120     while (static_cast<int64_t>(keys.size()) < entries_per_batch_) {
5121       key_guards.push_back(std::unique_ptr<const char[]>());
5122       keys.push_back(AllocateKey(&key_guards.back()));
5123     }
5124 
5125     Duration duration(FLAGS_duration, reads_);
5126     while (!duration.Done(1)) {
5127       DB* db = SelectDB(thread);
5128       if (FLAGS_multiread_stride) {
5129         int64_t key = GetRandomKey(&thread->rand);
5130         if ((key + (entries_per_batch_ - 1) * FLAGS_multiread_stride) >=
5131             static_cast<int64_t>(FLAGS_num)) {
5132           key = FLAGS_num - entries_per_batch_ * FLAGS_multiread_stride;
5133         }
5134         for (int64_t i = 0; i < entries_per_batch_; ++i) {
5135           GenerateKeyFromInt(key, FLAGS_num, &keys[i]);
5136           key += FLAGS_multiread_stride;
5137         }
5138       } else {
5139         for (int64_t i = 0; i < entries_per_batch_; ++i) {
5140           GenerateKeyFromInt(GetRandomKey(&thread->rand), FLAGS_num, &keys[i]);
5141         }
5142       }
5143       if (!FLAGS_multiread_batched) {
5144         std::vector<Status> statuses = db->MultiGet(options, keys, &values);
5145         assert(static_cast<int64_t>(statuses.size()) == entries_per_batch_);
5146 
5147         read += entries_per_batch_;
5148         num_multireads++;
5149         for (int64_t i = 0; i < entries_per_batch_; ++i) {
5150           if (statuses[i].ok()) {
5151             ++found;
5152           } else if (!statuses[i].IsNotFound()) {
5153             fprintf(stderr, "MultiGet returned an error: %s\n",
5154                     statuses[i].ToString().c_str());
5155             abort();
5156           }
5157         }
5158       } else {
5159         db->MultiGet(options, db->DefaultColumnFamily(), keys.size(),
5160                      keys.data(), pin_values, stat_list.data());
5161 
5162         read += entries_per_batch_;
5163         num_multireads++;
5164         for (int64_t i = 0; i < entries_per_batch_; ++i) {
5165           if (stat_list[i].ok()) {
5166             ++found;
5167           } else if (!stat_list[i].IsNotFound()) {
5168             fprintf(stderr, "MultiGet returned an error: %s\n",
5169                     stat_list[i].ToString().c_str());
5170             abort();
5171           }
5172           stat_list[i] = Status::OK();
5173           pin_values[i].Reset();
5174         }
5175       }
5176       if (thread->shared->read_rate_limiter.get() != nullptr &&
5177           num_multireads % 256 == 255) {
5178         thread->shared->read_rate_limiter->Request(
5179             256 * entries_per_batch_, Env::IO_HIGH, nullptr /* stats */,
5180             RateLimiter::OpType::kRead);
5181       }
5182       thread->stats.FinishedOps(nullptr, db, entries_per_batch_, kRead);
5183     }
5184 
5185     char msg[100];
5186     snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found)",
5187              found, read);
5188     thread->stats.AddMessage(msg);
5189   }
5190 
5191   // The inverse function of Pareto distribution
ParetoCdfInversion(double u,double theta,double k,double sigma)5192   int64_t ParetoCdfInversion(double u, double theta, double k, double sigma) {
5193     double ret;
5194     if (k == 0.0) {
5195       ret = theta - sigma * std::log(u);
5196     } else {
5197       ret = theta + sigma * (std::pow(u, -1 * k) - 1) / k;
5198     }
5199     return static_cast<int64_t>(ceil(ret));
5200   }
5201   // The inverse function of power distribution (y=ax^b)
PowerCdfInversion(double u,double a,double b)5202   int64_t PowerCdfInversion(double u, double a, double b) {
5203     double ret;
5204     ret = std::pow((u / a), (1 / b));
5205     return static_cast<int64_t>(ceil(ret));
5206   }
5207 
5208   // Add the noice to the QPS
AddNoise(double origin,double noise_ratio)5209   double AddNoise(double origin, double noise_ratio) {
5210     if (noise_ratio < 0.0 || noise_ratio > 1.0) {
5211       return origin;
5212     }
5213     int band_int = static_cast<int>(FLAGS_sine_a);
5214     double delta = (rand() % band_int - band_int / 2) * noise_ratio;
5215     if (origin + delta < 0) {
5216       return origin;
5217     } else {
5218       return (origin + delta);
5219     }
5220   }
5221 
5222   // Decide the ratio of different query types
5223   // 0 Get, 1 Put, 2 Seek, 3 SeekForPrev, 4 Delete, 5 SingleDelete, 6 merge
5224   class QueryDecider {
5225    public:
5226     std::vector<int> type_;
5227     std::vector<double> ratio_;
5228     int range_;
5229 
QueryDecider()5230     QueryDecider() {}
~QueryDecider()5231     ~QueryDecider() {}
5232 
Initiate(std::vector<double> ratio_input)5233     Status Initiate(std::vector<double> ratio_input) {
5234       int range_max = 1000;
5235       double sum = 0.0;
5236       for (auto& ratio : ratio_input) {
5237         sum += ratio;
5238       }
5239       range_ = 0;
5240       for (auto& ratio : ratio_input) {
5241         range_ += static_cast<int>(ceil(range_max * (ratio / sum)));
5242         type_.push_back(range_);
5243         ratio_.push_back(ratio / sum);
5244       }
5245       return Status::OK();
5246     }
5247 
GetType(int64_t rand_num)5248     int GetType(int64_t rand_num) {
5249       if (rand_num < 0) {
5250         rand_num = rand_num * (-1);
5251       }
5252       assert(range_ != 0);
5253       int pos = static_cast<int>(rand_num % range_);
5254       for (int i = 0; i < static_cast<int>(type_.size()); i++) {
5255         if (pos < type_[i]) {
5256           return i;
5257         }
5258       }
5259       return 0;
5260     }
5261   };
5262 
5263   // KeyrangeUnit is the struct of a keyrange. It is used in a keyrange vector
5264   // to transfer a random value to one keyrange based on the hotness.
5265   struct KeyrangeUnit {
5266     int64_t keyrange_start;
5267     int64_t keyrange_access;
5268     int64_t keyrange_keys;
5269   };
5270 
5271   // From our observations, the prefix hotness (key-range hotness) follows
5272   // the two-term-exponential distribution: f(x) = a*exp(b*x) + c*exp(d*x).
5273   // However, we cannot directly use the inverse function to decide a
5274   // key-range from a random distribution. To achieve it, we create a list of
5275   // KeyrangeUnit, each KeyrangeUnit occupies a range of integers whose size is
5276   // decided based on the hotness of the key-range. When a random value is
5277   // generated based on uniform distribution, we map it to the KeyrangeUnit Vec
5278   // and one KeyrangeUnit is selected. The probability of a  KeyrangeUnit being
5279   // selected is the same as the hotness of this KeyrangeUnit. After that, the
5280   // key can be randomly allocated to the key-range of this KeyrangeUnit, or we
5281   // can based on the power distribution (y=ax^b) to generate the offset of
5282   // the key in the selected key-range. In this way, we generate the keyID
5283   // based on the hotness of the prefix and also the key hotness distribution.
5284   class GenerateTwoTermExpKeys {
5285    public:
5286     int64_t keyrange_rand_max_;
5287     int64_t keyrange_size_;
5288     int64_t keyrange_num_;
5289     bool initiated_;
5290     std::vector<KeyrangeUnit> keyrange_set_;
5291 
GenerateTwoTermExpKeys()5292     GenerateTwoTermExpKeys() {
5293       keyrange_rand_max_ = FLAGS_num;
5294       initiated_ = false;
5295     }
5296 
~GenerateTwoTermExpKeys()5297     ~GenerateTwoTermExpKeys() {}
5298 
5299     // Initiate the KeyrangeUnit vector and calculate the size of each
5300     // KeyrangeUnit.
InitiateExpDistribution(int64_t total_keys,double prefix_a,double prefix_b,double prefix_c,double prefix_d)5301     Status InitiateExpDistribution(int64_t total_keys, double prefix_a,
5302                                    double prefix_b, double prefix_c,
5303                                    double prefix_d) {
5304       int64_t amplify = 0;
5305       int64_t keyrange_start = 0;
5306       initiated_ = true;
5307       if (FLAGS_keyrange_num <= 0) {
5308         keyrange_num_ = 1;
5309       } else {
5310         keyrange_num_ = FLAGS_keyrange_num;
5311       }
5312       keyrange_size_ = total_keys / keyrange_num_;
5313 
5314       // Calculate the key-range shares size based on the input parameters
5315       for (int64_t pfx = keyrange_num_; pfx >= 1; pfx--) {
5316         // Step 1. Calculate the probability that this key range will be
5317         // accessed in a query. It is based on the two-term expoential
5318         // distribution
5319         double keyrange_p = prefix_a * std::exp(prefix_b * pfx) +
5320                             prefix_c * std::exp(prefix_d * pfx);
5321         if (keyrange_p < std::pow(10.0, -16.0)) {
5322           keyrange_p = 0.0;
5323         }
5324         // Step 2. Calculate the amplify
5325         // In order to allocate a query to a key-range based on the random
5326         // number generated for this query, we need to extend the probability
5327         // of each key range from [0,1] to [0, amplify]. Amplify is calculated
5328         // by 1/(smallest key-range probability). In this way, we ensure that
5329         // all key-ranges are assigned with an Integer that  >=0
5330         if (amplify == 0 && keyrange_p > 0) {
5331           amplify = static_cast<int64_t>(std::floor(1 / keyrange_p)) + 1;
5332         }
5333 
5334         // Step 3. For each key-range, we calculate its position in the
5335         // [0, amplify] range, including the start, the size (keyrange_access)
5336         KeyrangeUnit p_unit;
5337         p_unit.keyrange_start = keyrange_start;
5338         if (0.0 >= keyrange_p) {
5339           p_unit.keyrange_access = 0;
5340         } else {
5341           p_unit.keyrange_access =
5342               static_cast<int64_t>(std::floor(amplify * keyrange_p));
5343         }
5344         p_unit.keyrange_keys = keyrange_size_;
5345         keyrange_set_.push_back(p_unit);
5346         keyrange_start += p_unit.keyrange_access;
5347       }
5348       keyrange_rand_max_ = keyrange_start;
5349 
5350       // Step 4. Shuffle the key-ranges randomly
5351       // Since the access probability is calculated from small to large,
5352       // If we do not re-allocate them, hot key-ranges are always at the end
5353       // and cold key-ranges are at the begin of the key space. Therefore, the
5354       // key-ranges are shuffled and the rand seed is only decide by the
5355       // key-range hotness distribution. With the same distribution parameters
5356       // the shuffle results are the same.
5357       Random64 rand_loca(keyrange_rand_max_);
5358       for (int64_t i = 0; i < FLAGS_keyrange_num; i++) {
5359         int64_t pos = rand_loca.Next() % FLAGS_keyrange_num;
5360         assert(i >= 0 && i < static_cast<int64_t>(keyrange_set_.size()) &&
5361                pos >= 0 && pos < static_cast<int64_t>(keyrange_set_.size()));
5362         std::swap(keyrange_set_[i], keyrange_set_[pos]);
5363       }
5364 
5365       // Step 5. Recalculate the prefix start postion after shuffling
5366       int64_t offset = 0;
5367       for (auto& p_unit : keyrange_set_) {
5368         p_unit.keyrange_start = offset;
5369         offset += p_unit.keyrange_access;
5370       }
5371 
5372       return Status::OK();
5373     }
5374 
5375     // Generate the Key ID according to the input ini_rand and key distribution
DistGetKeyID(int64_t ini_rand,double key_dist_a,double key_dist_b)5376     int64_t DistGetKeyID(int64_t ini_rand, double key_dist_a,
5377                          double key_dist_b) {
5378       int64_t keyrange_rand = ini_rand % keyrange_rand_max_;
5379 
5380       // Calculate and select one key-range that contains the new key
5381       int64_t start = 0, end = static_cast<int64_t>(keyrange_set_.size());
5382       while (start + 1 < end) {
5383         int64_t mid = start + (end - start) / 2;
5384         assert(mid >= 0 && mid < static_cast<int64_t>(keyrange_set_.size()));
5385         if (keyrange_rand < keyrange_set_[mid].keyrange_start) {
5386           end = mid;
5387         } else {
5388           start = mid;
5389         }
5390       }
5391       int64_t keyrange_id = start;
5392 
5393       // Select one key in the key-range and compose the keyID
5394       int64_t key_offset = 0, key_seed;
5395       if (key_dist_a == 0.0 && key_dist_b == 0.0) {
5396         key_offset = ini_rand % keyrange_size_;
5397       } else {
5398         key_seed = static_cast<int64_t>(
5399             ceil(std::pow((ini_rand / key_dist_a), (1 / key_dist_b))));
5400         Random64 rand_key(key_seed);
5401         key_offset = static_cast<int64_t>(rand_key.Next()) % keyrange_size_;
5402       }
5403       return keyrange_size_ * keyrange_id + key_offset;
5404     }
5405   };
5406 
5407   // The social graph wokrload mixed with Get, Put, Iterator queries.
5408   // The value size and iterator length follow Pareto distribution.
5409   // The overall key access follow power distribution. If user models the
5410   // workload based on different key-ranges (or different prefixes), user
5411   // can use two-term-exponential distribution to fit the workload. User
5412   // needs to decides the ratio between Get, Put, Iterator queries before
5413   // starting the benchmark.
MixGraph(ThreadState * thread)5414   void MixGraph(ThreadState* thread) {
5415     int64_t read = 0;  // including single gets and Next of iterators
5416     int64_t gets = 0;
5417     int64_t puts = 0;
5418     int64_t found = 0;
5419     int64_t seek = 0;
5420     int64_t seek_found = 0;
5421     int64_t bytes = 0;
5422     const int64_t default_value_max = 1 * 1024 * 1024;
5423     int64_t value_max = default_value_max;
5424     int64_t scan_len_max = FLAGS_mix_max_scan_len;
5425     double write_rate = 1000000.0;
5426     double read_rate = 1000000.0;
5427     bool use_prefix_modeling = false;
5428     GenerateTwoTermExpKeys gen_exp;
5429     std::vector<double> ratio{FLAGS_mix_get_ratio, FLAGS_mix_put_ratio,
5430                               FLAGS_mix_seek_ratio};
5431     char value_buffer[default_value_max];
5432     QueryDecider query;
5433     RandomGenerator gen;
5434     Status s;
5435     if (value_max > FLAGS_mix_max_value_size) {
5436       value_max = FLAGS_mix_max_value_size;
5437     }
5438 
5439     ReadOptions options(FLAGS_verify_checksum, true);
5440     std::unique_ptr<const char[]> key_guard;
5441     Slice key = AllocateKey(&key_guard);
5442     PinnableSlice pinnable_val;
5443     query.Initiate(ratio);
5444 
5445     // the limit of qps initiation
5446     if (FLAGS_sine_a != 0 || FLAGS_sine_d != 0) {
5447       thread->shared->read_rate_limiter.reset(NewGenericRateLimiter(
5448           static_cast<int64_t>(read_rate), 100000 /* refill_period_us */, 10 /* fairness */,
5449           RateLimiter::Mode::kReadsOnly));
5450       thread->shared->write_rate_limiter.reset(
5451           NewGenericRateLimiter(static_cast<int64_t>(write_rate)));
5452     }
5453 
5454     // Decide if user wants to use prefix based key generation
5455     if (FLAGS_keyrange_dist_a != 0.0 || FLAGS_keyrange_dist_b != 0.0 ||
5456         FLAGS_keyrange_dist_c != 0.0 || FLAGS_keyrange_dist_d != 0.0) {
5457       use_prefix_modeling = true;
5458       gen_exp.InitiateExpDistribution(
5459           FLAGS_num, FLAGS_keyrange_dist_a, FLAGS_keyrange_dist_b,
5460           FLAGS_keyrange_dist_c, FLAGS_keyrange_dist_d);
5461     }
5462 
5463     Duration duration(FLAGS_duration, reads_);
5464     while (!duration.Done(1)) {
5465       DBWithColumnFamilies* db_with_cfh = SelectDBWithCfh(thread);
5466       int64_t ini_rand, rand_v, key_rand, key_seed;
5467       ini_rand = GetRandomKey(&thread->rand);
5468       rand_v = ini_rand % FLAGS_num;
5469       double u = static_cast<double>(rand_v) / FLAGS_num;
5470 
5471       // Generate the keyID based on the key hotness and prefix hotness
5472       if (use_prefix_modeling) {
5473         key_rand =
5474             gen_exp.DistGetKeyID(ini_rand, FLAGS_key_dist_a, FLAGS_key_dist_b);
5475       } else {
5476         key_seed = PowerCdfInversion(u, FLAGS_key_dist_a, FLAGS_key_dist_b);
5477         Random64 rand(key_seed);
5478         key_rand = static_cast<int64_t>(rand.Next()) % FLAGS_num;
5479       }
5480       GenerateKeyFromInt(key_rand, FLAGS_num, &key);
5481       int query_type = query.GetType(rand_v);
5482 
5483       // change the qps
5484       uint64_t now = FLAGS_env->NowMicros();
5485       uint64_t usecs_since_last;
5486       if (now > thread->stats.GetSineInterval()) {
5487         usecs_since_last = now - thread->stats.GetSineInterval();
5488       } else {
5489         usecs_since_last = 0;
5490       }
5491 
5492       if (usecs_since_last >
5493           (FLAGS_sine_mix_rate_interval_milliseconds * uint64_t{1000})) {
5494         double usecs_since_start =
5495             static_cast<double>(now - thread->stats.GetStart());
5496         thread->stats.ResetSineInterval();
5497         double mix_rate_with_noise = AddNoise(
5498             SineRate(usecs_since_start / 1000000.0), FLAGS_sine_mix_rate_noise);
5499         read_rate = mix_rate_with_noise * (query.ratio_[0] + query.ratio_[2]);
5500         write_rate =
5501             mix_rate_with_noise * query.ratio_[1] * FLAGS_mix_ave_kv_size;
5502 
5503         thread->shared->write_rate_limiter.reset(
5504             NewGenericRateLimiter(static_cast<int64_t>(write_rate)));
5505         thread->shared->read_rate_limiter.reset(NewGenericRateLimiter(
5506             static_cast<int64_t>(read_rate),
5507             FLAGS_sine_mix_rate_interval_milliseconds * uint64_t{1000}, 10,
5508             RateLimiter::Mode::kReadsOnly));
5509       }
5510       // Start the query
5511       if (query_type == 0) {
5512         // the Get query
5513         gets++;
5514         read++;
5515         if (FLAGS_num_column_families > 1) {
5516           s = db_with_cfh->db->Get(options, db_with_cfh->GetCfh(key_rand), key,
5517                                    &pinnable_val);
5518         } else {
5519           pinnable_val.Reset();
5520           s = db_with_cfh->db->Get(options,
5521                                    db_with_cfh->db->DefaultColumnFamily(), key,
5522                                    &pinnable_val);
5523         }
5524 
5525         if (s.ok()) {
5526           found++;
5527           bytes += key.size() + pinnable_val.size();
5528         } else if (!s.IsNotFound()) {
5529           fprintf(stderr, "Get returned an error: %s\n", s.ToString().c_str());
5530           abort();
5531         }
5532 
5533         if (thread->shared->read_rate_limiter.get() != nullptr &&
5534             read % 256 == 255) {
5535           thread->shared->read_rate_limiter->Request(
5536               256, Env::IO_HIGH, nullptr /* stats */,
5537               RateLimiter::OpType::kRead);
5538         }
5539         thread->stats.FinishedOps(db_with_cfh, db_with_cfh->db, 1, kRead);
5540       } else if (query_type == 1) {
5541         // the Put query
5542         puts++;
5543         int64_t val_size = ParetoCdfInversion(
5544             u, FLAGS_value_theta, FLAGS_value_k, FLAGS_value_sigma);
5545         if (val_size < 0) {
5546           val_size = 10;
5547         } else if (val_size > value_max) {
5548           val_size = val_size % value_max;
5549         }
5550         s = db_with_cfh->db->Put(
5551             write_options_, key,
5552             gen.Generate(static_cast<unsigned int>(val_size)));
5553         if (!s.ok()) {
5554           fprintf(stderr, "put error: %s\n", s.ToString().c_str());
5555           exit(1);
5556         }
5557 
5558         if (thread->shared->write_rate_limiter) {
5559           thread->shared->write_rate_limiter->Request(
5560               key.size() + val_size, Env::IO_HIGH, nullptr /*stats*/,
5561               RateLimiter::OpType::kWrite);
5562         }
5563         thread->stats.FinishedOps(db_with_cfh, db_with_cfh->db, 1, kWrite);
5564       } else if (query_type == 2) {
5565         // Seek query
5566         if (db_with_cfh->db != nullptr) {
5567           Iterator* single_iter = nullptr;
5568           single_iter = db_with_cfh->db->NewIterator(options);
5569           if (single_iter != nullptr) {
5570             single_iter->Seek(key);
5571             seek++;
5572             read++;
5573             if (single_iter->Valid() && single_iter->key().compare(key) == 0) {
5574               seek_found++;
5575             }
5576             int64_t scan_length =
5577                 ParetoCdfInversion(u, FLAGS_iter_theta, FLAGS_iter_k,
5578                                    FLAGS_iter_sigma) %
5579                 scan_len_max;
5580             for (int64_t j = 0; j < scan_length && single_iter->Valid(); j++) {
5581               Slice value = single_iter->value();
5582               memcpy(value_buffer, value.data(),
5583                      std::min(value.size(), sizeof(value_buffer)));
5584               bytes += single_iter->key().size() + single_iter->value().size();
5585               single_iter->Next();
5586               assert(single_iter->status().ok());
5587             }
5588           }
5589           delete single_iter;
5590         }
5591         thread->stats.FinishedOps(db_with_cfh, db_with_cfh->db, 1, kSeek);
5592       }
5593     }
5594     char msg[256];
5595     snprintf(msg, sizeof(msg),
5596              "( Gets:%" PRIu64 " Puts:%" PRIu64 " Seek:%" PRIu64 " of %" PRIu64
5597              " in %" PRIu64 " found)\n",
5598              gets, puts, seek, found, read);
5599 
5600     thread->stats.AddBytes(bytes);
5601     thread->stats.AddMessage(msg);
5602 
5603     if (FLAGS_perf_level > ROCKSDB_NAMESPACE::PerfLevel::kDisable) {
5604       thread->stats.AddMessage(std::string("PERF_CONTEXT:\n") +
5605                                get_perf_context()->ToString());
5606     }
5607   }
5608 
IteratorCreation(ThreadState * thread)5609   void IteratorCreation(ThreadState* thread) {
5610     Duration duration(FLAGS_duration, reads_);
5611     ReadOptions options(FLAGS_verify_checksum, true);
5612     while (!duration.Done(1)) {
5613       DB* db = SelectDB(thread);
5614       Iterator* iter = db->NewIterator(options);
5615       delete iter;
5616       thread->stats.FinishedOps(nullptr, db, 1, kOthers);
5617     }
5618   }
5619 
IteratorCreationWhileWriting(ThreadState * thread)5620   void IteratorCreationWhileWriting(ThreadState* thread) {
5621     if (thread->tid > 0) {
5622       IteratorCreation(thread);
5623     } else {
5624       BGWriter(thread, kWrite);
5625     }
5626   }
5627 
SeekRandom(ThreadState * thread)5628   void SeekRandom(ThreadState* thread) {
5629     int64_t read = 0;
5630     int64_t found = 0;
5631     int64_t bytes = 0;
5632     ReadOptions options(FLAGS_verify_checksum, true);
5633     options.total_order_seek = FLAGS_total_order_seek;
5634     options.prefix_same_as_start = FLAGS_prefix_same_as_start;
5635     options.tailing = FLAGS_use_tailing_iterator;
5636     options.readahead_size = FLAGS_readahead_size;
5637 
5638     Iterator* single_iter = nullptr;
5639     std::vector<Iterator*> multi_iters;
5640     if (db_.db != nullptr) {
5641       single_iter = db_.db->NewIterator(options);
5642     } else {
5643       for (const auto& db_with_cfh : multi_dbs_) {
5644         multi_iters.push_back(db_with_cfh.db->NewIterator(options));
5645       }
5646     }
5647 
5648     std::unique_ptr<const char[]> key_guard;
5649     Slice key = AllocateKey(&key_guard);
5650 
5651     std::unique_ptr<const char[]> upper_bound_key_guard;
5652     Slice upper_bound = AllocateKey(&upper_bound_key_guard);
5653     std::unique_ptr<const char[]> lower_bound_key_guard;
5654     Slice lower_bound = AllocateKey(&lower_bound_key_guard);
5655 
5656     Duration duration(FLAGS_duration, reads_);
5657     char value_buffer[256];
5658     while (!duration.Done(1)) {
5659       int64_t seek_pos = thread->rand.Next() % FLAGS_num;
5660       GenerateKeyFromIntForSeek(static_cast<uint64_t>(seek_pos), FLAGS_num,
5661                                 &key);
5662       if (FLAGS_max_scan_distance != 0) {
5663         if (FLAGS_reverse_iterator) {
5664           GenerateKeyFromInt(
5665               static_cast<uint64_t>(std::max(
5666                   static_cast<int64_t>(0), seek_pos - FLAGS_max_scan_distance)),
5667               FLAGS_num, &lower_bound);
5668           options.iterate_lower_bound = &lower_bound;
5669         } else {
5670           auto min_num =
5671               std::min(FLAGS_num, seek_pos + FLAGS_max_scan_distance);
5672           GenerateKeyFromInt(static_cast<uint64_t>(min_num), FLAGS_num,
5673                              &upper_bound);
5674           options.iterate_upper_bound = &upper_bound;
5675         }
5676       }
5677 
5678       if (!FLAGS_use_tailing_iterator) {
5679         if (db_.db != nullptr) {
5680           delete single_iter;
5681           single_iter = db_.db->NewIterator(options);
5682         } else {
5683           for (auto iter : multi_iters) {
5684             delete iter;
5685           }
5686           multi_iters.clear();
5687           for (const auto& db_with_cfh : multi_dbs_) {
5688             multi_iters.push_back(db_with_cfh.db->NewIterator(options));
5689           }
5690         }
5691       }
5692       // Pick a Iterator to use
5693       Iterator* iter_to_use = single_iter;
5694       if (single_iter == nullptr) {
5695         iter_to_use = multi_iters[thread->rand.Next() % multi_iters.size()];
5696       }
5697 
5698       iter_to_use->Seek(key);
5699       read++;
5700       if (iter_to_use->Valid() && iter_to_use->key().compare(key) == 0) {
5701         found++;
5702       }
5703 
5704       for (int j = 0; j < FLAGS_seek_nexts && iter_to_use->Valid(); ++j) {
5705         // Copy out iterator's value to make sure we read them.
5706         Slice value = iter_to_use->value();
5707         memcpy(value_buffer, value.data(),
5708                std::min(value.size(), sizeof(value_buffer)));
5709         bytes += iter_to_use->key().size() + iter_to_use->value().size();
5710 
5711         if (!FLAGS_reverse_iterator) {
5712           iter_to_use->Next();
5713         } else {
5714           iter_to_use->Prev();
5715         }
5716         assert(iter_to_use->status().ok());
5717       }
5718 
5719       if (thread->shared->read_rate_limiter.get() != nullptr &&
5720           read % 256 == 255) {
5721         thread->shared->read_rate_limiter->Request(
5722             256, Env::IO_HIGH, nullptr /* stats */, RateLimiter::OpType::kRead);
5723       }
5724 
5725       thread->stats.FinishedOps(&db_, db_.db, 1, kSeek);
5726     }
5727     delete single_iter;
5728     for (auto iter : multi_iters) {
5729       delete iter;
5730     }
5731 
5732     char msg[100];
5733     snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found)\n",
5734              found, read);
5735     thread->stats.AddBytes(bytes);
5736     thread->stats.AddMessage(msg);
5737     if (FLAGS_perf_level > ROCKSDB_NAMESPACE::PerfLevel::kDisable) {
5738       thread->stats.AddMessage(std::string("PERF_CONTEXT:\n") +
5739                                get_perf_context()->ToString());
5740     }
5741   }
5742 
SeekRandomWhileWriting(ThreadState * thread)5743   void SeekRandomWhileWriting(ThreadState* thread) {
5744     if (thread->tid > 0) {
5745       SeekRandom(thread);
5746     } else {
5747       BGWriter(thread, kWrite);
5748     }
5749   }
5750 
SeekRandomWhileMerging(ThreadState * thread)5751   void SeekRandomWhileMerging(ThreadState* thread) {
5752     if (thread->tid > 0) {
5753       SeekRandom(thread);
5754     } else {
5755       BGWriter(thread, kMerge);
5756     }
5757   }
5758 
DoDelete(ThreadState * thread,bool seq)5759   void DoDelete(ThreadState* thread, bool seq) {
5760     WriteBatch batch;
5761     Duration duration(seq ? 0 : FLAGS_duration, deletes_);
5762     int64_t i = 0;
5763     std::unique_ptr<const char[]> key_guard;
5764     Slice key = AllocateKey(&key_guard);
5765 
5766     while (!duration.Done(entries_per_batch_)) {
5767       DB* db = SelectDB(thread);
5768       batch.Clear();
5769       for (int64_t j = 0; j < entries_per_batch_; ++j) {
5770         const int64_t k = seq ? i + j : (thread->rand.Next() % FLAGS_num);
5771         GenerateKeyFromInt(k, FLAGS_num, &key);
5772         batch.Delete(key);
5773       }
5774       auto s = db->Write(write_options_, &batch);
5775       thread->stats.FinishedOps(nullptr, db, entries_per_batch_, kDelete);
5776       if (!s.ok()) {
5777         fprintf(stderr, "del error: %s\n", s.ToString().c_str());
5778         exit(1);
5779       }
5780       i += entries_per_batch_;
5781     }
5782   }
5783 
DeleteSeq(ThreadState * thread)5784   void DeleteSeq(ThreadState* thread) {
5785     DoDelete(thread, true);
5786   }
5787 
DeleteRandom(ThreadState * thread)5788   void DeleteRandom(ThreadState* thread) {
5789     DoDelete(thread, false);
5790   }
5791 
ReadWhileWriting(ThreadState * thread)5792   void ReadWhileWriting(ThreadState* thread) {
5793     if (thread->tid > 0) {
5794       ReadRandom(thread);
5795     } else {
5796       BGWriter(thread, kWrite);
5797     }
5798   }
5799 
ReadWhileMerging(ThreadState * thread)5800   void ReadWhileMerging(ThreadState* thread) {
5801     if (thread->tid > 0) {
5802       ReadRandom(thread);
5803     } else {
5804       BGWriter(thread, kMerge);
5805     }
5806   }
5807 
BGWriter(ThreadState * thread,enum OperationType write_merge)5808   void BGWriter(ThreadState* thread, enum OperationType write_merge) {
5809     // Special thread that keeps writing until other threads are done.
5810     RandomGenerator gen;
5811     int64_t bytes = 0;
5812 
5813     std::unique_ptr<RateLimiter> write_rate_limiter;
5814     if (FLAGS_benchmark_write_rate_limit > 0) {
5815       write_rate_limiter.reset(
5816           NewGenericRateLimiter(FLAGS_benchmark_write_rate_limit));
5817     }
5818 
5819     // Don't merge stats from this thread with the readers.
5820     thread->stats.SetExcludeFromMerge();
5821 
5822     std::unique_ptr<const char[]> key_guard;
5823     Slice key = AllocateKey(&key_guard);
5824     uint32_t written = 0;
5825     bool hint_printed = false;
5826 
5827     while (true) {
5828       DB* db = SelectDB(thread);
5829       {
5830         MutexLock l(&thread->shared->mu);
5831         if (FLAGS_finish_after_writes && written == writes_) {
5832           fprintf(stderr, "Exiting the writer after %u writes...\n", written);
5833           break;
5834         }
5835         if (thread->shared->num_done + 1 >= thread->shared->num_initialized) {
5836           // Other threads have finished
5837           if (FLAGS_finish_after_writes) {
5838             // Wait for the writes to be finished
5839             if (!hint_printed) {
5840               fprintf(stderr, "Reads are finished. Have %d more writes to do\n",
5841                       static_cast<int>(writes_) - written);
5842               hint_printed = true;
5843             }
5844           } else {
5845             // Finish the write immediately
5846             break;
5847           }
5848         }
5849       }
5850 
5851       GenerateKeyFromInt(thread->rand.Next() % FLAGS_num, FLAGS_num, &key);
5852       Status s;
5853 
5854       Slice val = gen.Generate();
5855       if (write_merge == kWrite) {
5856         s = db->Put(write_options_, key, val);
5857       } else {
5858         s = db->Merge(write_options_, key, val);
5859       }
5860       written++;
5861 
5862       if (!s.ok()) {
5863         fprintf(stderr, "put or merge error: %s\n", s.ToString().c_str());
5864         exit(1);
5865       }
5866       bytes += key.size() + val.size();
5867       thread->stats.FinishedOps(&db_, db_.db, 1, kWrite);
5868 
5869       if (FLAGS_benchmark_write_rate_limit > 0) {
5870         write_rate_limiter->Request(
5871             key.size() + val.size(), Env::IO_HIGH,
5872             nullptr /* stats */, RateLimiter::OpType::kWrite);
5873       }
5874     }
5875     thread->stats.AddBytes(bytes);
5876   }
5877 
ReadWhileScanning(ThreadState * thread)5878   void ReadWhileScanning(ThreadState* thread) {
5879     if (thread->tid > 0) {
5880       ReadRandom(thread);
5881     } else {
5882       BGScan(thread);
5883     }
5884   }
5885 
BGScan(ThreadState * thread)5886   void BGScan(ThreadState* thread) {
5887     if (FLAGS_num_multi_db > 0) {
5888       fprintf(stderr, "Not supporting multiple DBs.\n");
5889       abort();
5890     }
5891     assert(db_.db != nullptr);
5892     ReadOptions read_options;
5893     Iterator* iter = db_.db->NewIterator(read_options);
5894 
5895     fprintf(stderr, "num reads to do %" PRIu64 "\n", reads_);
5896     Duration duration(FLAGS_duration, reads_);
5897     uint64_t num_seek_to_first = 0;
5898     uint64_t num_next = 0;
5899     while (!duration.Done(1)) {
5900       if (!iter->Valid()) {
5901         iter->SeekToFirst();
5902         num_seek_to_first++;
5903       } else if (!iter->status().ok()) {
5904         fprintf(stderr, "Iterator error: %s\n",
5905                 iter->status().ToString().c_str());
5906         abort();
5907       } else {
5908         iter->Next();
5909         num_next++;
5910       }
5911 
5912       thread->stats.FinishedOps(&db_, db_.db, 1, kSeek);
5913     }
5914     delete iter;
5915   }
5916 
5917   // Given a key K and value V, this puts (K+"0", V), (K+"1", V), (K+"2", V)
5918   // in DB atomically i.e in a single batch. Also refer GetMany.
PutMany(DB * db,const WriteOptions & writeoptions,const Slice & key,const Slice & value)5919   Status PutMany(DB* db, const WriteOptions& writeoptions, const Slice& key,
5920                  const Slice& value) {
5921     std::string suffixes[3] = {"2", "1", "0"};
5922     std::string keys[3];
5923 
5924     WriteBatch batch;
5925     Status s;
5926     for (int i = 0; i < 3; i++) {
5927       keys[i] = key.ToString() + suffixes[i];
5928       batch.Put(keys[i], value);
5929     }
5930 
5931     s = db->Write(writeoptions, &batch);
5932     return s;
5933   }
5934 
5935 
5936   // Given a key K, this deletes (K+"0", V), (K+"1", V), (K+"2", V)
5937   // in DB atomically i.e in a single batch. Also refer GetMany.
DeleteMany(DB * db,const WriteOptions & writeoptions,const Slice & key)5938   Status DeleteMany(DB* db, const WriteOptions& writeoptions,
5939                     const Slice& key) {
5940     std::string suffixes[3] = {"1", "2", "0"};
5941     std::string keys[3];
5942 
5943     WriteBatch batch;
5944     Status s;
5945     for (int i = 0; i < 3; i++) {
5946       keys[i] = key.ToString() + suffixes[i];
5947       batch.Delete(keys[i]);
5948     }
5949 
5950     s = db->Write(writeoptions, &batch);
5951     return s;
5952   }
5953 
5954   // Given a key K and value V, this gets values for K+"0", K+"1" and K+"2"
5955   // in the same snapshot, and verifies that all the values are identical.
5956   // ASSUMES that PutMany was used to put (K, V) into the DB.
GetMany(DB * db,const ReadOptions & readoptions,const Slice & key,std::string * value)5957   Status GetMany(DB* db, const ReadOptions& readoptions, const Slice& key,
5958                  std::string* value) {
5959     std::string suffixes[3] = {"0", "1", "2"};
5960     std::string keys[3];
5961     Slice key_slices[3];
5962     std::string values[3];
5963     ReadOptions readoptionscopy = readoptions;
5964     readoptionscopy.snapshot = db->GetSnapshot();
5965     Status s;
5966     for (int i = 0; i < 3; i++) {
5967       keys[i] = key.ToString() + suffixes[i];
5968       key_slices[i] = keys[i];
5969       s = db->Get(readoptionscopy, key_slices[i], value);
5970       if (!s.ok() && !s.IsNotFound()) {
5971         fprintf(stderr, "get error: %s\n", s.ToString().c_str());
5972         values[i] = "";
5973         // we continue after error rather than exiting so that we can
5974         // find more errors if any
5975       } else if (s.IsNotFound()) {
5976         values[i] = "";
5977       } else {
5978         values[i] = *value;
5979       }
5980     }
5981     db->ReleaseSnapshot(readoptionscopy.snapshot);
5982 
5983     if ((values[0] != values[1]) || (values[1] != values[2])) {
5984       fprintf(stderr, "inconsistent values for key %s: %s, %s, %s\n",
5985               key.ToString().c_str(), values[0].c_str(), values[1].c_str(),
5986               values[2].c_str());
5987       // we continue after error rather than exiting so that we can
5988       // find more errors if any
5989     }
5990 
5991     return s;
5992   }
5993 
5994   // Differs from readrandomwriterandom in the following ways:
5995   // (a) Uses GetMany/PutMany to read/write key values. Refer to those funcs.
5996   // (b) Does deletes as well (per FLAGS_deletepercent)
5997   // (c) In order to achieve high % of 'found' during lookups, and to do
5998   //     multiple writes (including puts and deletes) it uses upto
5999   //     FLAGS_numdistinct distinct keys instead of FLAGS_num distinct keys.
6000   // (d) Does not have a MultiGet option.
RandomWithVerify(ThreadState * thread)6001   void RandomWithVerify(ThreadState* thread) {
6002     ReadOptions options(FLAGS_verify_checksum, true);
6003     RandomGenerator gen;
6004     std::string value;
6005     int64_t found = 0;
6006     int get_weight = 0;
6007     int put_weight = 0;
6008     int delete_weight = 0;
6009     int64_t gets_done = 0;
6010     int64_t puts_done = 0;
6011     int64_t deletes_done = 0;
6012 
6013     std::unique_ptr<const char[]> key_guard;
6014     Slice key = AllocateKey(&key_guard);
6015 
6016     // the number of iterations is the larger of read_ or write_
6017     for (int64_t i = 0; i < readwrites_; i++) {
6018       DB* db = SelectDB(thread);
6019       if (get_weight == 0 && put_weight == 0 && delete_weight == 0) {
6020         // one batch completed, reinitialize for next batch
6021         get_weight = FLAGS_readwritepercent;
6022         delete_weight = FLAGS_deletepercent;
6023         put_weight = 100 - get_weight - delete_weight;
6024       }
6025       GenerateKeyFromInt(thread->rand.Next() % FLAGS_numdistinct,
6026           FLAGS_numdistinct, &key);
6027       if (get_weight > 0) {
6028         // do all the gets first
6029         Status s = GetMany(db, options, key, &value);
6030         if (!s.ok() && !s.IsNotFound()) {
6031           fprintf(stderr, "getmany error: %s\n", s.ToString().c_str());
6032           // we continue after error rather than exiting so that we can
6033           // find more errors if any
6034         } else if (!s.IsNotFound()) {
6035           found++;
6036         }
6037         get_weight--;
6038         gets_done++;
6039         thread->stats.FinishedOps(&db_, db_.db, 1, kRead);
6040       } else if (put_weight > 0) {
6041         // then do all the corresponding number of puts
6042         // for all the gets we have done earlier
6043         Status s = PutMany(db, write_options_, key, gen.Generate());
6044         if (!s.ok()) {
6045           fprintf(stderr, "putmany error: %s\n", s.ToString().c_str());
6046           exit(1);
6047         }
6048         put_weight--;
6049         puts_done++;
6050         thread->stats.FinishedOps(&db_, db_.db, 1, kWrite);
6051       } else if (delete_weight > 0) {
6052         Status s = DeleteMany(db, write_options_, key);
6053         if (!s.ok()) {
6054           fprintf(stderr, "deletemany error: %s\n", s.ToString().c_str());
6055           exit(1);
6056         }
6057         delete_weight--;
6058         deletes_done++;
6059         thread->stats.FinishedOps(&db_, db_.db, 1, kDelete);
6060       }
6061     }
6062     char msg[128];
6063     snprintf(msg, sizeof(msg),
6064              "( get:%" PRIu64 " put:%" PRIu64 " del:%" PRIu64 " total:%" \
6065              PRIu64 " found:%" PRIu64 ")",
6066              gets_done, puts_done, deletes_done, readwrites_, found);
6067     thread->stats.AddMessage(msg);
6068   }
6069 
6070   // This is different from ReadWhileWriting because it does not use
6071   // an extra thread.
ReadRandomWriteRandom(ThreadState * thread)6072   void ReadRandomWriteRandom(ThreadState* thread) {
6073     ReadOptions options(FLAGS_verify_checksum, true);
6074     RandomGenerator gen;
6075     std::string value;
6076     int64_t found = 0;
6077     int get_weight = 0;
6078     int put_weight = 0;
6079     int64_t reads_done = 0;
6080     int64_t writes_done = 0;
6081     Duration duration(FLAGS_duration, readwrites_);
6082 
6083     std::unique_ptr<const char[]> key_guard;
6084     Slice key = AllocateKey(&key_guard);
6085 
6086     // the number of iterations is the larger of read_ or write_
6087     while (!duration.Done(1)) {
6088       DB* db = SelectDB(thread);
6089       GenerateKeyFromInt(thread->rand.Next() % FLAGS_num, FLAGS_num, &key);
6090       if (get_weight == 0 && put_weight == 0) {
6091         // one batch completed, reinitialize for next batch
6092         get_weight = FLAGS_readwritepercent;
6093         put_weight = 100 - get_weight;
6094       }
6095       if (get_weight > 0) {
6096         // do all the gets first
6097         Status s = db->Get(options, key, &value);
6098         if (!s.ok() && !s.IsNotFound()) {
6099           fprintf(stderr, "get error: %s\n", s.ToString().c_str());
6100           // we continue after error rather than exiting so that we can
6101           // find more errors if any
6102         } else if (!s.IsNotFound()) {
6103           found++;
6104         }
6105         get_weight--;
6106         reads_done++;
6107         thread->stats.FinishedOps(nullptr, db, 1, kRead);
6108       } else  if (put_weight > 0) {
6109         // then do all the corresponding number of puts
6110         // for all the gets we have done earlier
6111         Status s = db->Put(write_options_, key, gen.Generate());
6112         if (!s.ok()) {
6113           fprintf(stderr, "put error: %s\n", s.ToString().c_str());
6114           exit(1);
6115         }
6116         put_weight--;
6117         writes_done++;
6118         thread->stats.FinishedOps(nullptr, db, 1, kWrite);
6119       }
6120     }
6121     char msg[100];
6122     snprintf(msg, sizeof(msg), "( reads:%" PRIu64 " writes:%" PRIu64 \
6123              " total:%" PRIu64 " found:%" PRIu64 ")",
6124              reads_done, writes_done, readwrites_, found);
6125     thread->stats.AddMessage(msg);
6126   }
6127 
6128   //
6129   // Read-modify-write for random keys
UpdateRandom(ThreadState * thread)6130   void UpdateRandom(ThreadState* thread) {
6131     ReadOptions options(FLAGS_verify_checksum, true);
6132     RandomGenerator gen;
6133     std::string value;
6134     int64_t found = 0;
6135     int64_t bytes = 0;
6136     Duration duration(FLAGS_duration, readwrites_);
6137 
6138     std::unique_ptr<const char[]> key_guard;
6139     Slice key = AllocateKey(&key_guard);
6140     // the number of iterations is the larger of read_ or write_
6141     while (!duration.Done(1)) {
6142       DB* db = SelectDB(thread);
6143       GenerateKeyFromInt(thread->rand.Next() % FLAGS_num, FLAGS_num, &key);
6144 
6145       auto status = db->Get(options, key, &value);
6146       if (status.ok()) {
6147         ++found;
6148         bytes += key.size() + value.size();
6149       } else if (!status.IsNotFound()) {
6150         fprintf(stderr, "Get returned an error: %s\n",
6151                 status.ToString().c_str());
6152         abort();
6153       }
6154 
6155       if (thread->shared->write_rate_limiter) {
6156         thread->shared->write_rate_limiter->Request(
6157             key.size() + value.size(), Env::IO_HIGH, nullptr /*stats*/,
6158             RateLimiter::OpType::kWrite);
6159       }
6160 
6161       Slice val = gen.Generate();
6162       Status s = db->Put(write_options_, key, val);
6163       if (!s.ok()) {
6164         fprintf(stderr, "put error: %s\n", s.ToString().c_str());
6165         exit(1);
6166       }
6167       bytes += key.size() + val.size();
6168       thread->stats.FinishedOps(nullptr, db, 1, kUpdate);
6169     }
6170     char msg[100];
6171     snprintf(msg, sizeof(msg),
6172              "( updates:%" PRIu64 " found:%" PRIu64 ")", readwrites_, found);
6173     thread->stats.AddBytes(bytes);
6174     thread->stats.AddMessage(msg);
6175   }
6176 
6177   // Read-XOR-write for random keys. Xors the existing value with a randomly
6178   // generated value, and stores the result. Assuming A in the array of bytes
6179   // representing the existing value, we generate an array B of the same size,
6180   // then compute C = A^B as C[i]=A[i]^B[i], and store C
XORUpdateRandom(ThreadState * thread)6181   void XORUpdateRandom(ThreadState* thread) {
6182     ReadOptions options(FLAGS_verify_checksum, true);
6183     RandomGenerator gen;
6184     std::string existing_value;
6185     int64_t found = 0;
6186     Duration duration(FLAGS_duration, readwrites_);
6187 
6188     BytesXOROperator xor_operator;
6189 
6190     std::unique_ptr<const char[]> key_guard;
6191     Slice key = AllocateKey(&key_guard);
6192     // the number of iterations is the larger of read_ or write_
6193     while (!duration.Done(1)) {
6194       DB* db = SelectDB(thread);
6195       GenerateKeyFromInt(thread->rand.Next() % FLAGS_num, FLAGS_num, &key);
6196 
6197       auto status = db->Get(options, key, &existing_value);
6198       if (status.ok()) {
6199         ++found;
6200       } else if (!status.IsNotFound()) {
6201         fprintf(stderr, "Get returned an error: %s\n",
6202                 status.ToString().c_str());
6203         exit(1);
6204       }
6205 
6206       Slice value = gen.Generate(static_cast<unsigned int>(existing_value.size()));
6207       std::string new_value;
6208 
6209       if (status.ok()) {
6210         Slice existing_value_slice = Slice(existing_value);
6211         xor_operator.XOR(&existing_value_slice, value, &new_value);
6212       } else {
6213         xor_operator.XOR(nullptr, value, &new_value);
6214       }
6215 
6216       Status s = db->Put(write_options_, key, Slice(new_value));
6217       if (!s.ok()) {
6218         fprintf(stderr, "put error: %s\n", s.ToString().c_str());
6219         exit(1);
6220       }
6221       thread->stats.FinishedOps(nullptr, db, 1);
6222     }
6223     char msg[100];
6224     snprintf(msg, sizeof(msg),
6225              "( updates:%" PRIu64 " found:%" PRIu64 ")", readwrites_, found);
6226     thread->stats.AddMessage(msg);
6227   }
6228 
6229   // Read-modify-write for random keys.
6230   // Each operation causes the key grow by value_size (simulating an append).
6231   // Generally used for benchmarking against merges of similar type
AppendRandom(ThreadState * thread)6232   void AppendRandom(ThreadState* thread) {
6233     ReadOptions options(FLAGS_verify_checksum, true);
6234     RandomGenerator gen;
6235     std::string value;
6236     int64_t found = 0;
6237     int64_t bytes = 0;
6238 
6239     std::unique_ptr<const char[]> key_guard;
6240     Slice key = AllocateKey(&key_guard);
6241     // The number of iterations is the larger of read_ or write_
6242     Duration duration(FLAGS_duration, readwrites_);
6243     while (!duration.Done(1)) {
6244       DB* db = SelectDB(thread);
6245       GenerateKeyFromInt(thread->rand.Next() % FLAGS_num, FLAGS_num, &key);
6246 
6247       auto status = db->Get(options, key, &value);
6248       if (status.ok()) {
6249         ++found;
6250         bytes += key.size() + value.size();
6251       } else if (!status.IsNotFound()) {
6252         fprintf(stderr, "Get returned an error: %s\n",
6253                 status.ToString().c_str());
6254         abort();
6255       } else {
6256         // If not existing, then just assume an empty string of data
6257         value.clear();
6258       }
6259 
6260       // Update the value (by appending data)
6261       Slice operand = gen.Generate();
6262       if (value.size() > 0) {
6263         // Use a delimiter to match the semantics for StringAppendOperator
6264         value.append(1,',');
6265       }
6266       value.append(operand.data(), operand.size());
6267 
6268       // Write back to the database
6269       Status s = db->Put(write_options_, key, value);
6270       if (!s.ok()) {
6271         fprintf(stderr, "put error: %s\n", s.ToString().c_str());
6272         exit(1);
6273       }
6274       bytes += key.size() + value.size();
6275       thread->stats.FinishedOps(nullptr, db, 1, kUpdate);
6276     }
6277 
6278     char msg[100];
6279     snprintf(msg, sizeof(msg), "( updates:%" PRIu64 " found:%" PRIu64 ")",
6280             readwrites_, found);
6281     thread->stats.AddBytes(bytes);
6282     thread->stats.AddMessage(msg);
6283   }
6284 
6285   // Read-modify-write for random keys (using MergeOperator)
6286   // The merge operator to use should be defined by FLAGS_merge_operator
6287   // Adjust FLAGS_value_size so that the keys are reasonable for this operator
6288   // Assumes that the merge operator is non-null (i.e.: is well-defined)
6289   //
6290   // For example, use FLAGS_merge_operator="uint64add" and FLAGS_value_size=8
6291   // to simulate random additions over 64-bit integers using merge.
6292   //
6293   // The number of merges on the same key can be controlled by adjusting
6294   // FLAGS_merge_keys.
MergeRandom(ThreadState * thread)6295   void MergeRandom(ThreadState* thread) {
6296     RandomGenerator gen;
6297     int64_t bytes = 0;
6298     std::unique_ptr<const char[]> key_guard;
6299     Slice key = AllocateKey(&key_guard);
6300     // The number of iterations is the larger of read_ or write_
6301     Duration duration(FLAGS_duration, readwrites_);
6302     while (!duration.Done(1)) {
6303       DBWithColumnFamilies* db_with_cfh = SelectDBWithCfh(thread);
6304       int64_t key_rand = thread->rand.Next() % merge_keys_;
6305       GenerateKeyFromInt(key_rand, merge_keys_, &key);
6306 
6307       Status s;
6308       Slice val = gen.Generate();
6309       if (FLAGS_num_column_families > 1) {
6310         s = db_with_cfh->db->Merge(write_options_,
6311                                    db_with_cfh->GetCfh(key_rand), key,
6312                                    val);
6313       } else {
6314         s = db_with_cfh->db->Merge(write_options_,
6315                                    db_with_cfh->db->DefaultColumnFamily(), key,
6316                                    val);
6317       }
6318 
6319       if (!s.ok()) {
6320         fprintf(stderr, "merge error: %s\n", s.ToString().c_str());
6321         exit(1);
6322       }
6323       bytes += key.size() + val.size();
6324       thread->stats.FinishedOps(nullptr, db_with_cfh->db, 1, kMerge);
6325     }
6326 
6327     // Print some statistics
6328     char msg[100];
6329     snprintf(msg, sizeof(msg), "( updates:%" PRIu64 ")", readwrites_);
6330     thread->stats.AddBytes(bytes);
6331     thread->stats.AddMessage(msg);
6332   }
6333 
6334   // Read and merge random keys. The amount of reads and merges are controlled
6335   // by adjusting FLAGS_num and FLAGS_mergereadpercent. The number of distinct
6336   // keys (and thus also the number of reads and merges on the same key) can be
6337   // adjusted with FLAGS_merge_keys.
6338   //
6339   // As with MergeRandom, the merge operator to use should be defined by
6340   // FLAGS_merge_operator.
ReadRandomMergeRandom(ThreadState * thread)6341   void ReadRandomMergeRandom(ThreadState* thread) {
6342     ReadOptions options(FLAGS_verify_checksum, true);
6343     RandomGenerator gen;
6344     std::string value;
6345     int64_t num_hits = 0;
6346     int64_t num_gets = 0;
6347     int64_t num_merges = 0;
6348     size_t max_length = 0;
6349 
6350     std::unique_ptr<const char[]> key_guard;
6351     Slice key = AllocateKey(&key_guard);
6352     // the number of iterations is the larger of read_ or write_
6353     Duration duration(FLAGS_duration, readwrites_);
6354     while (!duration.Done(1)) {
6355       DB* db = SelectDB(thread);
6356       GenerateKeyFromInt(thread->rand.Next() % merge_keys_, merge_keys_, &key);
6357 
6358       bool do_merge = int(thread->rand.Next() % 100) < FLAGS_mergereadpercent;
6359 
6360       if (do_merge) {
6361         Status s = db->Merge(write_options_, key, gen.Generate());
6362         if (!s.ok()) {
6363           fprintf(stderr, "merge error: %s\n", s.ToString().c_str());
6364           exit(1);
6365         }
6366         num_merges++;
6367         thread->stats.FinishedOps(nullptr, db, 1, kMerge);
6368       } else {
6369         Status s = db->Get(options, key, &value);
6370         if (value.length() > max_length)
6371           max_length = value.length();
6372 
6373         if (!s.ok() && !s.IsNotFound()) {
6374           fprintf(stderr, "get error: %s\n", s.ToString().c_str());
6375           // we continue after error rather than exiting so that we can
6376           // find more errors if any
6377         } else if (!s.IsNotFound()) {
6378           num_hits++;
6379         }
6380         num_gets++;
6381         thread->stats.FinishedOps(nullptr, db, 1, kRead);
6382       }
6383     }
6384 
6385     char msg[100];
6386     snprintf(msg, sizeof(msg),
6387              "(reads:%" PRIu64 " merges:%" PRIu64 " total:%" PRIu64
6388              " hits:%" PRIu64 " maxlength:%" ROCKSDB_PRIszt ")",
6389              num_gets, num_merges, readwrites_, num_hits, max_length);
6390     thread->stats.AddMessage(msg);
6391   }
6392 
WriteSeqSeekSeq(ThreadState * thread)6393   void WriteSeqSeekSeq(ThreadState* thread) {
6394     writes_ = FLAGS_num;
6395     DoWrite(thread, SEQUENTIAL);
6396     // exclude writes from the ops/sec calculation
6397     thread->stats.Start(thread->tid);
6398 
6399     DB* db = SelectDB(thread);
6400     std::unique_ptr<Iterator> iter(
6401       db->NewIterator(ReadOptions(FLAGS_verify_checksum, true)));
6402 
6403     std::unique_ptr<const char[]> key_guard;
6404     Slice key = AllocateKey(&key_guard);
6405     for (int64_t i = 0; i < FLAGS_num; ++i) {
6406       GenerateKeyFromInt(i, FLAGS_num, &key);
6407       iter->Seek(key);
6408       assert(iter->Valid() && iter->key() == key);
6409       thread->stats.FinishedOps(nullptr, db, 1, kSeek);
6410 
6411       for (int j = 0; j < FLAGS_seek_nexts && i + 1 < FLAGS_num; ++j) {
6412         if (!FLAGS_reverse_iterator) {
6413           iter->Next();
6414         } else {
6415           iter->Prev();
6416         }
6417         GenerateKeyFromInt(++i, FLAGS_num, &key);
6418         assert(iter->Valid() && iter->key() == key);
6419         thread->stats.FinishedOps(nullptr, db, 1, kSeek);
6420       }
6421 
6422       iter->Seek(key);
6423       assert(iter->Valid() && iter->key() == key);
6424       thread->stats.FinishedOps(nullptr, db, 1, kSeek);
6425     }
6426   }
6427 
binary_search(std::vector<int> & data,int start,int end,int key)6428   bool binary_search(std::vector<int>& data, int start, int end, int key) {
6429     if (data.empty()) return false;
6430     if (start > end) return false;
6431     int mid = start + (end - start) / 2;
6432     if (mid > static_cast<int>(data.size()) - 1) return false;
6433     if (data[mid] == key) {
6434       return true;
6435     } else if (data[mid] > key) {
6436       return binary_search(data, start, mid - 1, key);
6437     } else {
6438       return binary_search(data, mid + 1, end, key);
6439     }
6440   }
6441 
6442   // Does a bunch of merge operations for a key(key1) where the merge operand
6443   // is a sorted list. Next performance comparison is done between doing a Get
6444   // for key1 followed by searching for another key(key2) in the large sorted
6445   // list vs calling GetMergeOperands for key1 and then searching for the key2
6446   // in all the sorted sub-lists. Later case is expected to be a lot faster.
GetMergeOperands(ThreadState * thread)6447   void GetMergeOperands(ThreadState* thread) {
6448     DB* db = SelectDB(thread);
6449     const int kTotalValues = 100000;
6450     const int kListSize = 100;
6451     std::string key = "my_key";
6452     std::string value;
6453 
6454     for (int i = 1; i < kTotalValues; i++) {
6455       if (i % kListSize == 0) {
6456         // Remove trailing ','
6457         value.pop_back();
6458         db->Merge(WriteOptions(), key, value);
6459         value.clear();
6460       } else {
6461         value.append(std::to_string(i)).append(",");
6462       }
6463     }
6464 
6465     SortList s;
6466     std::vector<int> data;
6467     // This value can be experimented with and it will demonstrate the
6468     // perf difference between doing a Get and searching for lookup_key in the
6469     // resultant large sorted list vs doing GetMergeOperands and searching
6470     // for lookup_key within this resultant sorted sub-lists.
6471     int lookup_key = 1;
6472 
6473     // Get API call
6474     std::cout << "--- Get API call --- \n";
6475     PinnableSlice p_slice;
6476     uint64_t st = FLAGS_env->NowNanos();
6477     db->Get(ReadOptions(), db->DefaultColumnFamily(), key, &p_slice);
6478     s.MakeVector(data, p_slice);
6479     bool found =
6480         binary_search(data, 0, static_cast<int>(data.size() - 1), lookup_key);
6481     std::cout << "Found key? " << std::to_string(found) << "\n";
6482     uint64_t sp = FLAGS_env->NowNanos();
6483     std::cout << "Get: " << (sp - st) / 1000000000.0 << " seconds\n";
6484     std::string* dat_ = p_slice.GetSelf();
6485     std::cout << "Sample data from Get API call: " << dat_->substr(0, 10)
6486               << "\n";
6487     data.clear();
6488 
6489     // GetMergeOperands API call
6490     std::cout << "--- GetMergeOperands API --- \n";
6491     std::vector<PinnableSlice> a_slice((kTotalValues / kListSize) + 1);
6492     st = FLAGS_env->NowNanos();
6493     int number_of_operands = 0;
6494     GetMergeOperandsOptions get_merge_operands_options;
6495     get_merge_operands_options.expected_max_number_of_operands =
6496         (kTotalValues / 100) + 1;
6497     db->GetMergeOperands(ReadOptions(), db->DefaultColumnFamily(), key,
6498                          a_slice.data(), &get_merge_operands_options,
6499                          &number_of_operands);
6500     for (PinnableSlice& psl : a_slice) {
6501       s.MakeVector(data, psl);
6502       found =
6503           binary_search(data, 0, static_cast<int>(data.size() - 1), lookup_key);
6504       data.clear();
6505       if (found) break;
6506     }
6507     std::cout << "Found key? " << std::to_string(found) << "\n";
6508     sp = FLAGS_env->NowNanos();
6509     std::cout << "Get Merge operands: " << (sp - st) / 1000000000.0
6510               << " seconds \n";
6511     int to_print = 0;
6512     std::cout << "Sample data from GetMergeOperands API call: ";
6513     for (PinnableSlice& psl : a_slice) {
6514       std::cout << "List: " << to_print << " : " << *psl.GetSelf() << "\n";
6515       if (to_print++ > 2) break;
6516     }
6517   }
6518 
6519 #ifndef ROCKSDB_LITE
6520   // This benchmark stress tests Transactions.  For a given --duration (or
6521   // total number of --writes, a Transaction will perform a read-modify-write
6522   // to increment the value of a key in each of N(--transaction-sets) sets of
6523   // keys (where each set has --num keys).  If --threads is set, this will be
6524   // done in parallel.
6525   //
6526   // To test transactions, use --transaction_db=true.  Not setting this
6527   // parameter
6528   // will run the same benchmark without transactions.
6529   //
6530   // RandomTransactionVerify() will then validate the correctness of the results
6531   // by checking if the sum of all keys in each set is the same.
RandomTransaction(ThreadState * thread)6532   void RandomTransaction(ThreadState* thread) {
6533     ReadOptions options(FLAGS_verify_checksum, true);
6534     Duration duration(FLAGS_duration, readwrites_);
6535     ReadOptions read_options(FLAGS_verify_checksum, true);
6536     uint16_t num_prefix_ranges = static_cast<uint16_t>(FLAGS_transaction_sets);
6537     uint64_t transactions_done = 0;
6538 
6539     if (num_prefix_ranges == 0 || num_prefix_ranges > 9999) {
6540       fprintf(stderr, "invalid value for transaction_sets\n");
6541       abort();
6542     }
6543 
6544     TransactionOptions txn_options;
6545     txn_options.lock_timeout = FLAGS_transaction_lock_timeout;
6546     txn_options.set_snapshot = FLAGS_transaction_set_snapshot;
6547 
6548     RandomTransactionInserter inserter(&thread->rand, write_options_,
6549                                        read_options, FLAGS_num,
6550                                        num_prefix_ranges);
6551 
6552     if (FLAGS_num_multi_db > 1) {
6553       fprintf(stderr,
6554               "Cannot run RandomTransaction benchmark with "
6555               "FLAGS_multi_db > 1.");
6556       abort();
6557     }
6558 
6559     while (!duration.Done(1)) {
6560       bool success;
6561 
6562       // RandomTransactionInserter will attempt to insert a key for each
6563       // # of FLAGS_transaction_sets
6564       if (FLAGS_optimistic_transaction_db) {
6565         success = inserter.OptimisticTransactionDBInsert(db_.opt_txn_db);
6566       } else if (FLAGS_transaction_db) {
6567         TransactionDB* txn_db = reinterpret_cast<TransactionDB*>(db_.db);
6568         success = inserter.TransactionDBInsert(txn_db, txn_options);
6569       } else {
6570         success = inserter.DBInsert(db_.db);
6571       }
6572 
6573       if (!success) {
6574         fprintf(stderr, "Unexpected error: %s\n",
6575                 inserter.GetLastStatus().ToString().c_str());
6576         abort();
6577       }
6578 
6579       thread->stats.FinishedOps(nullptr, db_.db, 1, kOthers);
6580       transactions_done++;
6581     }
6582 
6583     char msg[100];
6584     if (FLAGS_optimistic_transaction_db || FLAGS_transaction_db) {
6585       snprintf(msg, sizeof(msg),
6586                "( transactions:%" PRIu64 " aborts:%" PRIu64 ")",
6587                transactions_done, inserter.GetFailureCount());
6588     } else {
6589       snprintf(msg, sizeof(msg), "( batches:%" PRIu64 " )", transactions_done);
6590     }
6591     thread->stats.AddMessage(msg);
6592 
6593     if (FLAGS_perf_level > ROCKSDB_NAMESPACE::PerfLevel::kDisable) {
6594       thread->stats.AddMessage(std::string("PERF_CONTEXT:\n") +
6595                                get_perf_context()->ToString());
6596     }
6597     thread->stats.AddBytes(static_cast<int64_t>(inserter.GetBytesInserted()));
6598   }
6599 
6600   // Verifies consistency of data after RandomTransaction() has been run.
6601   // Since each iteration of RandomTransaction() incremented a key in each set
6602   // by the same value, the sum of the keys in each set should be the same.
RandomTransactionVerify()6603   void RandomTransactionVerify() {
6604     if (!FLAGS_transaction_db && !FLAGS_optimistic_transaction_db) {
6605       // transactions not used, nothing to verify.
6606       return;
6607     }
6608 
6609     Status s =
6610         RandomTransactionInserter::Verify(db_.db,
6611                             static_cast<uint16_t>(FLAGS_transaction_sets));
6612 
6613     if (s.ok()) {
6614       fprintf(stdout, "RandomTransactionVerify Success.\n");
6615     } else {
6616       fprintf(stdout, "RandomTransactionVerify FAILED!!\n");
6617     }
6618   }
6619 #endif  // ROCKSDB_LITE
6620 
6621   // Writes and deletes random keys without overwriting keys.
6622   //
6623   // This benchmark is intended to partially replicate the behavior of MyRocks
6624   // secondary indices: All data is stored in keys and updates happen by
6625   // deleting the old version of the key and inserting the new version.
RandomReplaceKeys(ThreadState * thread)6626   void RandomReplaceKeys(ThreadState* thread) {
6627     std::unique_ptr<const char[]> key_guard;
6628     Slice key = AllocateKey(&key_guard);
6629     std::vector<uint32_t> counters(FLAGS_numdistinct, 0);
6630     size_t max_counter = 50;
6631     RandomGenerator gen;
6632 
6633     Status s;
6634     DB* db = SelectDB(thread);
6635     for (int64_t i = 0; i < FLAGS_numdistinct; i++) {
6636       GenerateKeyFromInt(i * max_counter, FLAGS_num, &key);
6637       s = db->Put(write_options_, key, gen.Generate());
6638       if (!s.ok()) {
6639         fprintf(stderr, "Operation failed: %s\n", s.ToString().c_str());
6640         exit(1);
6641       }
6642     }
6643 
6644     db->GetSnapshot();
6645 
6646     std::default_random_engine generator;
6647     std::normal_distribution<double> distribution(FLAGS_numdistinct / 2.0,
6648                                                   FLAGS_stddev);
6649     Duration duration(FLAGS_duration, FLAGS_num);
6650     while (!duration.Done(1)) {
6651       int64_t rnd_id = static_cast<int64_t>(distribution(generator));
6652       int64_t key_id = std::max(std::min(FLAGS_numdistinct - 1, rnd_id),
6653                                 static_cast<int64_t>(0));
6654       GenerateKeyFromInt(key_id * max_counter + counters[key_id], FLAGS_num,
6655                          &key);
6656       s = FLAGS_use_single_deletes ? db->SingleDelete(write_options_, key)
6657                                    : db->Delete(write_options_, key);
6658       if (s.ok()) {
6659         counters[key_id] = (counters[key_id] + 1) % max_counter;
6660         GenerateKeyFromInt(key_id * max_counter + counters[key_id], FLAGS_num,
6661                            &key);
6662         s = db->Put(write_options_, key, Slice());
6663       }
6664 
6665       if (!s.ok()) {
6666         fprintf(stderr, "Operation failed: %s\n", s.ToString().c_str());
6667         exit(1);
6668       }
6669 
6670       thread->stats.FinishedOps(nullptr, db, 1, kOthers);
6671     }
6672 
6673     char msg[200];
6674     snprintf(msg, sizeof(msg),
6675              "use single deletes: %d, "
6676              "standard deviation: %lf\n",
6677              FLAGS_use_single_deletes, FLAGS_stddev);
6678     thread->stats.AddMessage(msg);
6679   }
6680 
TimeSeriesReadOrDelete(ThreadState * thread,bool do_deletion)6681   void TimeSeriesReadOrDelete(ThreadState* thread, bool do_deletion) {
6682     ReadOptions options(FLAGS_verify_checksum, true);
6683     int64_t read = 0;
6684     int64_t found = 0;
6685     int64_t bytes = 0;
6686 
6687     Iterator* iter = nullptr;
6688     // Only work on single database
6689     assert(db_.db != nullptr);
6690     iter = db_.db->NewIterator(options);
6691 
6692     std::unique_ptr<const char[]> key_guard;
6693     Slice key = AllocateKey(&key_guard);
6694 
6695     char value_buffer[256];
6696     while (true) {
6697       {
6698         MutexLock l(&thread->shared->mu);
6699         if (thread->shared->num_done >= 1) {
6700           // Write thread have finished
6701           break;
6702         }
6703       }
6704       if (!FLAGS_use_tailing_iterator) {
6705         delete iter;
6706         iter = db_.db->NewIterator(options);
6707       }
6708       // Pick a Iterator to use
6709 
6710       int64_t key_id = thread->rand.Next() % FLAGS_key_id_range;
6711       GenerateKeyFromInt(key_id, FLAGS_num, &key);
6712       // Reset last 8 bytes to 0
6713       char* start = const_cast<char*>(key.data());
6714       start += key.size() - 8;
6715       memset(start, 0, 8);
6716       ++read;
6717 
6718       bool key_found = false;
6719       // Seek the prefix
6720       for (iter->Seek(key); iter->Valid() && iter->key().starts_with(key);
6721            iter->Next()) {
6722         key_found = true;
6723         // Copy out iterator's value to make sure we read them.
6724         if (do_deletion) {
6725           bytes += iter->key().size();
6726           if (KeyExpired(timestamp_emulator_.get(), iter->key())) {
6727             thread->stats.FinishedOps(&db_, db_.db, 1, kDelete);
6728             db_.db->Delete(write_options_, iter->key());
6729           } else {
6730             break;
6731           }
6732         } else {
6733           bytes += iter->key().size() + iter->value().size();
6734           thread->stats.FinishedOps(&db_, db_.db, 1, kRead);
6735           Slice value = iter->value();
6736           memcpy(value_buffer, value.data(),
6737                  std::min(value.size(), sizeof(value_buffer)));
6738 
6739           assert(iter->status().ok());
6740         }
6741       }
6742       found += key_found;
6743 
6744       if (thread->shared->read_rate_limiter.get() != nullptr) {
6745         thread->shared->read_rate_limiter->Request(
6746             1, Env::IO_HIGH, nullptr /* stats */, RateLimiter::OpType::kRead);
6747       }
6748     }
6749     delete iter;
6750 
6751     char msg[100];
6752     snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found)", found,
6753              read);
6754     thread->stats.AddBytes(bytes);
6755     thread->stats.AddMessage(msg);
6756     if (FLAGS_perf_level > ROCKSDB_NAMESPACE::PerfLevel::kDisable) {
6757       thread->stats.AddMessage(std::string("PERF_CONTEXT:\n") +
6758                                get_perf_context()->ToString());
6759     }
6760   }
6761 
TimeSeriesWrite(ThreadState * thread)6762   void TimeSeriesWrite(ThreadState* thread) {
6763     // Special thread that keeps writing until other threads are done.
6764     RandomGenerator gen;
6765     int64_t bytes = 0;
6766 
6767     // Don't merge stats from this thread with the readers.
6768     thread->stats.SetExcludeFromMerge();
6769 
6770     std::unique_ptr<RateLimiter> write_rate_limiter;
6771     if (FLAGS_benchmark_write_rate_limit > 0) {
6772       write_rate_limiter.reset(
6773           NewGenericRateLimiter(FLAGS_benchmark_write_rate_limit));
6774     }
6775 
6776     std::unique_ptr<const char[]> key_guard;
6777     Slice key = AllocateKey(&key_guard);
6778 
6779     Duration duration(FLAGS_duration, writes_);
6780     while (!duration.Done(1)) {
6781       DB* db = SelectDB(thread);
6782 
6783       uint64_t key_id = thread->rand.Next() % FLAGS_key_id_range;
6784       // Write key id
6785       GenerateKeyFromInt(key_id, FLAGS_num, &key);
6786       // Write timestamp
6787 
6788       char* start = const_cast<char*>(key.data());
6789       char* pos = start + 8;
6790       int bytes_to_fill =
6791           std::min(key_size_ - static_cast<int>(pos - start), 8);
6792       uint64_t timestamp_value = timestamp_emulator_->Get();
6793       if (port::kLittleEndian) {
6794         for (int i = 0; i < bytes_to_fill; ++i) {
6795           pos[i] = (timestamp_value >> ((bytes_to_fill - i - 1) << 3)) & 0xFF;
6796         }
6797       } else {
6798         memcpy(pos, static_cast<void*>(&timestamp_value), bytes_to_fill);
6799       }
6800 
6801       timestamp_emulator_->Inc();
6802 
6803       Status s;
6804       Slice val = gen.Generate();
6805       s = db->Put(write_options_, key, val);
6806 
6807       if (!s.ok()) {
6808         fprintf(stderr, "put error: %s\n", s.ToString().c_str());
6809         exit(1);
6810       }
6811       bytes = key.size() + val.size();
6812       thread->stats.FinishedOps(&db_, db_.db, 1, kWrite);
6813       thread->stats.AddBytes(bytes);
6814 
6815       if (FLAGS_benchmark_write_rate_limit > 0) {
6816         write_rate_limiter->Request(
6817             key.size() + val.size(), Env::IO_HIGH,
6818             nullptr /* stats */, RateLimiter::OpType::kWrite);
6819       }
6820     }
6821   }
6822 
TimeSeries(ThreadState * thread)6823   void TimeSeries(ThreadState* thread) {
6824     if (thread->tid > 0) {
6825       bool do_deletion = FLAGS_expire_style == "delete" &&
6826                          thread->tid <= FLAGS_num_deletion_threads;
6827       TimeSeriesReadOrDelete(thread, do_deletion);
6828     } else {
6829       TimeSeriesWrite(thread);
6830       thread->stats.Stop();
6831       thread->stats.Report("timeseries write");
6832     }
6833   }
6834 
Compact(ThreadState * thread)6835   void Compact(ThreadState* thread) {
6836     DB* db = SelectDB(thread);
6837     CompactRangeOptions cro;
6838     cro.bottommost_level_compaction =
6839         BottommostLevelCompaction::kForceOptimized;
6840     db->CompactRange(cro, nullptr, nullptr);
6841   }
6842 
CompactAll()6843   void CompactAll() {
6844     if (db_.db != nullptr) {
6845       db_.db->CompactRange(CompactRangeOptions(), nullptr, nullptr);
6846     }
6847     for (const auto& db_with_cfh : multi_dbs_) {
6848       db_with_cfh.db->CompactRange(CompactRangeOptions(), nullptr, nullptr);
6849     }
6850   }
6851 
ResetStats()6852   void ResetStats() {
6853     if (db_.db != nullptr) {
6854       db_.db->ResetStats();
6855     }
6856     for (const auto& db_with_cfh : multi_dbs_) {
6857       db_with_cfh.db->ResetStats();
6858     }
6859   }
6860 
PrintStatsHistory()6861   void PrintStatsHistory() {
6862     if (db_.db != nullptr) {
6863       PrintStatsHistoryImpl(db_.db, false);
6864     }
6865     for (const auto& db_with_cfh : multi_dbs_) {
6866       PrintStatsHistoryImpl(db_with_cfh.db, true);
6867     }
6868   }
6869 
PrintStatsHistoryImpl(DB * db,bool print_header)6870   void PrintStatsHistoryImpl(DB* db, bool print_header) {
6871     if (print_header) {
6872       fprintf(stdout, "\n==== DB: %s ===\n", db->GetName().c_str());
6873     }
6874 
6875     std::unique_ptr<StatsHistoryIterator> shi;
6876     Status s = db->GetStatsHistory(0, port::kMaxUint64, &shi);
6877     if (!s.ok()) {
6878       fprintf(stdout, "%s\n", s.ToString().c_str());
6879       return;
6880     }
6881     assert(shi);
6882     while (shi->Valid()) {
6883       uint64_t stats_time = shi->GetStatsTime();
6884       fprintf(stdout, "------ %s ------\n",
6885               TimeToHumanString(static_cast<int>(stats_time)).c_str());
6886       for (auto& entry : shi->GetStatsMap()) {
6887         fprintf(stdout, " %" PRIu64 "   %s  %" PRIu64 "\n", stats_time,
6888                 entry.first.c_str(), entry.second);
6889       }
6890       shi->Next();
6891     }
6892   }
6893 
PrintStats(const char * key)6894   void PrintStats(const char* key) {
6895     if (db_.db != nullptr) {
6896       PrintStats(db_.db, key, false);
6897     }
6898     for (const auto& db_with_cfh : multi_dbs_) {
6899       PrintStats(db_with_cfh.db, key, true);
6900     }
6901   }
6902 
PrintStats(DB * db,const char * key,bool print_header=false)6903   void PrintStats(DB* db, const char* key, bool print_header = false) {
6904     if (print_header) {
6905       fprintf(stdout, "\n==== DB: %s ===\n", db->GetName().c_str());
6906     }
6907     std::string stats;
6908     if (!db->GetProperty(key, &stats)) {
6909       stats = "(failed)";
6910     }
6911     fprintf(stdout, "\n%s\n", stats.c_str());
6912   }
6913 
Replay(ThreadState * thread)6914   void Replay(ThreadState* thread) {
6915     if (db_.db != nullptr) {
6916       Replay(thread, &db_);
6917     }
6918   }
6919 
Replay(ThreadState *,DBWithColumnFamilies * db_with_cfh)6920   void Replay(ThreadState* /*thread*/, DBWithColumnFamilies* db_with_cfh) {
6921     Status s;
6922     std::unique_ptr<TraceReader> trace_reader;
6923     s = NewFileTraceReader(FLAGS_env, EnvOptions(), FLAGS_trace_file,
6924                            &trace_reader);
6925     if (!s.ok()) {
6926       fprintf(
6927           stderr,
6928           "Encountered an error creating a TraceReader from the trace file. "
6929           "Error: %s\n",
6930           s.ToString().c_str());
6931       exit(1);
6932     }
6933     Replayer replayer(db_with_cfh->db, db_with_cfh->cfh,
6934                       std::move(trace_reader));
6935     replayer.SetFastForward(
6936         static_cast<uint32_t>(FLAGS_trace_replay_fast_forward));
6937     s = replayer.MultiThreadReplay(
6938         static_cast<uint32_t>(FLAGS_trace_replay_threads));
6939     if (s.ok()) {
6940       fprintf(stdout, "Replay started from trace_file: %s\n",
6941               FLAGS_trace_file.c_str());
6942     } else {
6943       fprintf(stderr, "Starting replay failed. Error: %s\n",
6944               s.ToString().c_str());
6945     }
6946   }
6947 };
6948 
db_bench_tool(int argc,char ** argv)6949 int db_bench_tool(int argc, char** argv) {
6950   ROCKSDB_NAMESPACE::port::InstallStackTraceHandler();
6951   static bool initialized = false;
6952   if (!initialized) {
6953     SetUsageMessage(std::string("\nUSAGE:\n") + std::string(argv[0]) +
6954                     " [OPTIONS]...");
6955     initialized = true;
6956   }
6957   ParseCommandLineFlags(&argc, &argv, true);
6958   FLAGS_compaction_style_e =
6959       (ROCKSDB_NAMESPACE::CompactionStyle)FLAGS_compaction_style;
6960 #ifndef ROCKSDB_LITE
6961   if (FLAGS_statistics && !FLAGS_statistics_string.empty()) {
6962     fprintf(stderr,
6963             "Cannot provide both --statistics and --statistics_string.\n");
6964     exit(1);
6965   }
6966   if (!FLAGS_statistics_string.empty()) {
6967     Status s = ObjectRegistry::NewInstance()->NewSharedObject<Statistics>(
6968         FLAGS_statistics_string, &dbstats);
6969     if (dbstats == nullptr) {
6970       fprintf(stderr,
6971               "No Statistics registered matching string: %s status=%s\n",
6972               FLAGS_statistics_string.c_str(), s.ToString().c_str());
6973       exit(1);
6974     }
6975   }
6976 #endif  // ROCKSDB_LITE
6977   if (FLAGS_statistics) {
6978     dbstats = ROCKSDB_NAMESPACE::CreateDBStatistics();
6979   }
6980   if (dbstats) {
6981     dbstats->set_stats_level(static_cast<StatsLevel>(FLAGS_stats_level));
6982   }
6983   FLAGS_compaction_pri_e =
6984       (ROCKSDB_NAMESPACE::CompactionPri)FLAGS_compaction_pri;
6985 
6986   std::vector<std::string> fanout = ROCKSDB_NAMESPACE::StringSplit(
6987       FLAGS_max_bytes_for_level_multiplier_additional, ',');
6988   for (size_t j = 0; j < fanout.size(); j++) {
6989     FLAGS_max_bytes_for_level_multiplier_additional_v.push_back(
6990 #ifndef CYGWIN
6991         std::stoi(fanout[j]));
6992 #else
6993         stoi(fanout[j]));
6994 #endif
6995   }
6996 
6997   FLAGS_compression_type_e =
6998     StringToCompressionType(FLAGS_compression_type.c_str());
6999 
7000 #ifndef ROCKSDB_LITE
7001   FLAGS_blob_db_compression_type_e =
7002     StringToCompressionType(FLAGS_blob_db_compression_type.c_str());
7003 
7004   if (!FLAGS_hdfs.empty() && !FLAGS_env_uri.empty()) {
7005     fprintf(stderr, "Cannot provide both --hdfs and --env_uri.\n");
7006     exit(1);
7007   } else if (!FLAGS_env_uri.empty()) {
7008     Status s = Env::LoadEnv(FLAGS_env_uri, &FLAGS_env, &env_guard);
7009     if (FLAGS_env == nullptr) {
7010       fprintf(stderr, "No Env registered for URI: %s\n", FLAGS_env_uri.c_str());
7011       exit(1);
7012     }
7013   }
7014 #endif  // ROCKSDB_LITE
7015   if (FLAGS_use_existing_keys && !FLAGS_use_existing_db) {
7016     fprintf(stderr,
7017             "`-use_existing_db` must be true for `-use_existing_keys` to be "
7018             "settable\n");
7019     exit(1);
7020   }
7021 
7022   if (!FLAGS_hdfs.empty()) {
7023     FLAGS_env = new ROCKSDB_NAMESPACE::HdfsEnv(FLAGS_hdfs);
7024   }
7025 
7026   if (!strcasecmp(FLAGS_compaction_fadvice.c_str(), "NONE"))
7027     FLAGS_compaction_fadvice_e = ROCKSDB_NAMESPACE::Options::NONE;
7028   else if (!strcasecmp(FLAGS_compaction_fadvice.c_str(), "NORMAL"))
7029     FLAGS_compaction_fadvice_e = ROCKSDB_NAMESPACE::Options::NORMAL;
7030   else if (!strcasecmp(FLAGS_compaction_fadvice.c_str(), "SEQUENTIAL"))
7031     FLAGS_compaction_fadvice_e = ROCKSDB_NAMESPACE::Options::SEQUENTIAL;
7032   else if (!strcasecmp(FLAGS_compaction_fadvice.c_str(), "WILLNEED"))
7033     FLAGS_compaction_fadvice_e = ROCKSDB_NAMESPACE::Options::WILLNEED;
7034   else {
7035     fprintf(stdout, "Unknown compaction fadvice:%s\n",
7036             FLAGS_compaction_fadvice.c_str());
7037   }
7038 
7039   FLAGS_value_size_distribution_type_e =
7040     StringToDistributionType(FLAGS_value_size_distribution_type.c_str());
7041 
7042   FLAGS_rep_factory = StringToRepFactory(FLAGS_memtablerep.c_str());
7043 
7044   // Note options sanitization may increase thread pool sizes according to
7045   // max_background_flushes/max_background_compactions/max_background_jobs
7046   FLAGS_env->SetBackgroundThreads(FLAGS_num_high_pri_threads,
7047                                   ROCKSDB_NAMESPACE::Env::Priority::HIGH);
7048   FLAGS_env->SetBackgroundThreads(FLAGS_num_bottom_pri_threads,
7049                                   ROCKSDB_NAMESPACE::Env::Priority::BOTTOM);
7050   FLAGS_env->SetBackgroundThreads(FLAGS_num_low_pri_threads,
7051                                   ROCKSDB_NAMESPACE::Env::Priority::LOW);
7052 
7053   // Choose a location for the test database if none given with --db=<path>
7054   if (FLAGS_db.empty()) {
7055     std::string default_db_path;
7056     FLAGS_env->GetTestDirectory(&default_db_path);
7057     default_db_path += "/dbbench";
7058     FLAGS_db = default_db_path;
7059   }
7060 
7061   if (FLAGS_stats_interval_seconds > 0) {
7062     // When both are set then FLAGS_stats_interval determines the frequency
7063     // at which the timer is checked for FLAGS_stats_interval_seconds
7064     FLAGS_stats_interval = 1000;
7065   }
7066 
7067   if (FLAGS_seek_missing_prefix && FLAGS_prefix_size <= 8) {
7068     fprintf(stderr, "prefix_size > 8 required by --seek_missing_prefix\n");
7069     exit(1);
7070   }
7071 
7072   ROCKSDB_NAMESPACE::Benchmark benchmark;
7073   benchmark.Run();
7074 
7075 #ifndef ROCKSDB_LITE
7076   if (FLAGS_print_malloc_stats) {
7077     std::string stats_string;
7078     ROCKSDB_NAMESPACE::DumpMallocStats(&stats_string);
7079     fprintf(stdout, "Malloc stats:\n%s\n", stats_string.c_str());
7080   }
7081 #endif  // ROCKSDB_LITE
7082 
7083   return 0;
7084 }
7085 }  // namespace ROCKSDB_NAMESPACE
7086 #endif
7087