2013-10-16 23:59:46 +02:00
|
|
|
// Copyright (c) 2013, Facebook, Inc. All rights reserved.
|
|
|
|
// This source code is licensed under the BSD-style license found in the
|
|
|
|
// LICENSE file in the root directory of this source tree. An additional grant
|
|
|
|
// of patent rights can be found in the PATENTS file in the same directory.
|
|
|
|
//
|
2011-03-18 23:37:00 +01:00
|
|
|
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
|
|
|
// Use of this source code is governed by a BSD-style license that can be
|
|
|
|
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
|
|
|
|
2014-09-05 08:14:37 +02:00
|
|
|
#ifndef __STDC_FORMAT_MACROS
|
2014-03-05 02:08:05 +01:00
|
|
|
#define __STDC_FORMAT_MACROS
|
2014-09-05 08:14:37 +02:00
|
|
|
#endif
|
2014-05-09 02:25:13 +02:00
|
|
|
|
|
|
|
#ifndef GFLAGS
|
|
|
|
#include <cstdio>
|
|
|
|
int main() {
|
|
|
|
fprintf(stderr, "Please install gflags to run rocksdb tools\n");
|
|
|
|
return 1;
|
|
|
|
}
|
|
|
|
#else
|
|
|
|
|
2014-07-07 19:53:31 +02:00
|
|
|
#ifdef NUMA
|
|
|
|
#include <numa.h>
|
|
|
|
#include <numaif.h>
|
|
|
|
#endif
|
|
|
|
|
2015-07-02 01:13:49 +02:00
|
|
|
#ifndef OS_WIN
|
2015-04-07 02:16:40 +02:00
|
|
|
#include <unistd.h>
|
2015-07-02 01:13:49 +02:00
|
|
|
#endif
|
2015-04-03 03:14:49 +02:00
|
|
|
#include <fcntl.h>
|
2014-03-05 02:08:05 +01:00
|
|
|
#include <inttypes.h>
|
2013-02-15 20:53:17 +01:00
|
|
|
#include <cstddef>
|
2011-03-18 23:37:00 +01:00
|
|
|
#include <sys/types.h>
|
|
|
|
#include <stdio.h>
|
|
|
|
#include <stdlib.h>
|
2013-10-24 16:43:14 +02:00
|
|
|
#include <gflags/gflags.h>
|
db_bench periodically writes QPS to CSV file
Summary:
This is part of an effort to better understand and optimize RocksDB stalls under high load. I added a feature to db_bench to periodically write QPS to CSV files. That way we can nicely see how our QPS changes in time (especially when DB is stalled) and can do a better job of evaluating our stall system (i.e. we want the QPS to be as constant as possible, as opposed to having bunch of stalls)
Cool part of CSV files is that we can easily graph them -- there are a bunch of tools available.
Test Plan:
Ran ./db_bench --report_interval_seconds=10 --benchmarks=fillrandom --num=10000000
and observed this in report.csv:
secs_elapsed,interval_qps
10,2725860
20,1980480
30,1863456
40,1454359
50,1460389
Reviewers: sdong, MarkCallaghan, rven, yhchiang
Reviewed By: yhchiang
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D40047
2015-06-12 23:31:53 +02:00
|
|
|
|
|
|
|
#include <atomic>
|
|
|
|
#include <condition_variable>
|
|
|
|
#include <mutex>
|
|
|
|
#include <thread>
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
#include "db/db_impl.h"
|
|
|
|
#include "db/version_set.h"
|
2013-08-23 17:38:13 +02:00
|
|
|
#include "rocksdb/options.h"
|
|
|
|
#include "rocksdb/cache.h"
|
|
|
|
#include "rocksdb/db.h"
|
|
|
|
#include "rocksdb/env.h"
|
2013-09-17 23:11:04 +02:00
|
|
|
#include "rocksdb/memtablerep.h"
|
2013-08-23 17:38:13 +02:00
|
|
|
#include "rocksdb/write_batch.h"
|
2014-01-25 01:15:05 +01:00
|
|
|
#include "rocksdb/slice.h"
|
2014-09-08 19:37:05 +02:00
|
|
|
#include "rocksdb/filter_policy.h"
|
2015-05-13 02:40:27 +02:00
|
|
|
#include "rocksdb/rate_limiter.h"
|
2014-01-25 01:15:05 +01:00
|
|
|
#include "rocksdb/slice_transform.h"
|
2013-11-18 20:32:54 +01:00
|
|
|
#include "rocksdb/perf_context.h"
|
2015-03-30 18:51:11 +02:00
|
|
|
#include "rocksdb/utilities/flashcache.h"
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
#include "rocksdb/utilities/transaction.h"
|
|
|
|
#include "rocksdb/utilities/transaction_db.h"
|
2015-05-29 23:36:35 +02:00
|
|
|
#include "rocksdb/utilities/optimistic_transaction_db.h"
|
2011-03-25 21:27:43 +01:00
|
|
|
#include "port/port.h"
|
2014-04-23 15:11:35 +02:00
|
|
|
#include "port/stack_trace.h"
|
2011-03-25 21:27:43 +01:00
|
|
|
#include "util/crc32c.h"
|
2015-01-09 22:04:06 +01:00
|
|
|
#include "util/compression.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
#include "util/histogram.h"
|
2011-08-22 23:08:51 +02:00
|
|
|
#include "util/mutexlock.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
#include "util/random.h"
|
2013-05-21 20:37:06 +02:00
|
|
|
#include "util/string_util.h"
|
2014-01-17 21:46:06 +01:00
|
|
|
#include "util/statistics.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
#include "util/testutil.h"
|
2014-05-01 20:09:32 +02:00
|
|
|
#include "util/xxhash.h"
|
2012-06-08 10:11:14 +02:00
|
|
|
#include "hdfs/env_hdfs.h"
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
#include "utilities/merge_operators.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2015-07-02 01:13:49 +02:00
|
|
|
#ifdef OS_WIN
|
2015-07-13 21:11:05 +02:00
|
|
|
#include <io.h> // open/close
|
2015-07-02 01:13:49 +02:00
|
|
|
#endif
|
|
|
|
|
2014-05-09 02:25:13 +02:00
|
|
|
using GFLAGS::ParseCommandLineFlags;
|
|
|
|
using GFLAGS::RegisterFlagValidator;
|
|
|
|
using GFLAGS::SetUsageMessage;
|
2013-08-23 23:49:57 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_string(benchmarks,
|
|
|
|
"fillseq,"
|
|
|
|
"fillsync,"
|
|
|
|
"fillrandom,"
|
|
|
|
"overwrite,"
|
|
|
|
"readrandom,"
|
2014-04-10 19:15:59 +02:00
|
|
|
"newiterator,"
|
|
|
|
"newiteratorwhilewriting,"
|
2014-04-11 18:47:20 +02:00
|
|
|
"seekrandom,"
|
|
|
|
"seekrandomwhilewriting,"
|
2015-03-30 20:28:25 +02:00
|
|
|
"seekrandomwhilemerging,"
|
2013-10-24 16:43:14 +02:00
|
|
|
"readseq,"
|
|
|
|
"readreverse,"
|
|
|
|
"compact,"
|
|
|
|
"readrandom,"
|
2014-04-08 20:21:09 +02:00
|
|
|
"multireadrandom,"
|
2013-10-24 16:43:14 +02:00
|
|
|
"readseq,"
|
2013-12-19 01:50:48 +01:00
|
|
|
"readtocache,"
|
2013-10-24 16:43:14 +02:00
|
|
|
"readreverse,"
|
|
|
|
"readwhilewriting,"
|
2015-03-18 21:50:52 +01:00
|
|
|
"readwhilemerging,"
|
2013-10-24 16:43:14 +02:00
|
|
|
"readrandomwriterandom,"
|
|
|
|
"updaterandom,"
|
|
|
|
"randomwithverify,"
|
|
|
|
"fill100K,"
|
|
|
|
"crc32c,"
|
2014-05-01 20:09:32 +02:00
|
|
|
"xxhash,"
|
2014-02-08 03:12:30 +01:00
|
|
|
"compress,"
|
|
|
|
"uncompress,"
|
SkipListRep::LookaheadIterator
Summary:
This diff introduces the `lookahead` argument to `SkipListFactory()`. This is an
optimization for the tailing use case which includes many seeks. E.g. consider
the following operations on a skip list iterator:
Seek(x), Next(), Next(), Seek(x+2), Next(), Seek(x+3), Next(), Next(), ...
If `lookahead` is positive, `SkipListRep` will return an iterator which also
keeps track of the previously visited node. Seek() then first does a linear
search starting from that node (up to `lookahead` steps). As in the tailing
example above, this may require fewer than ~log(n) comparisons as with regular
skip list search.
Test Plan:
Added a new benchmark (`fillseekseq`) which simulates the usage pattern. It
first writes N records (with consecutive keys), then measures how much time it
takes to read them by calling `Seek()` and `Next()`.
$ time ./db_bench -num 10000000 -benchmarks fillseekseq -prefix_size 1 \
-key_size 8 -write_buffer_size $[1024*1024*1024] -value_size 50 \
-seekseq_next 2 -skip_list_lookahead=0
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.389 micros/op 2569047 ops/sec;
real 0m21.806s
user 0m12.106s
sys 0m9.672s
$ time ./db_bench [...] -skip_list_lookahead=2
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.153 micros/op 6540684 ops/sec;
real 0m19.469s
user 0m10.192s
sys 0m9.252s
Reviewers: ljin, sdong, igor
Reviewed By: igor
Subscribers: dhruba, leveldb, march, lovro
Differential Revision: https://reviews.facebook.net/D23997
2014-09-24 00:52:28 +02:00
|
|
|
"acquireload,"
|
2015-05-29 23:36:35 +02:00
|
|
|
"fillseekseq,"
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
"randomtransaction,"
|
|
|
|
"randomreplacekeys",
|
2013-10-24 16:43:14 +02:00
|
|
|
|
2015-08-11 20:46:15 +02:00
|
|
|
"Comma-separated list of operations to run in the specified"
|
|
|
|
" order. Available benchmarks:\n"
|
2013-10-24 16:43:14 +02:00
|
|
|
"\tfillseq -- write N values in sequential key"
|
|
|
|
" order in async mode\n"
|
|
|
|
"\tfillrandom -- write N values in random key order in async"
|
|
|
|
" mode\n"
|
|
|
|
"\toverwrite -- overwrite N values in random key order in"
|
|
|
|
" async mode\n"
|
|
|
|
"\tfillsync -- write N/100 values in random key order in "
|
|
|
|
"sync mode\n"
|
|
|
|
"\tfill100K -- write N/1000 100K values in random order in"
|
|
|
|
" async mode\n"
|
|
|
|
"\tdeleteseq -- delete N keys in sequential order\n"
|
|
|
|
"\tdeleterandom -- delete N keys in random order\n"
|
|
|
|
"\treadseq -- read N times sequentially\n"
|
2013-12-19 01:50:48 +01:00
|
|
|
"\treadtocache -- 1 thread reading database sequentially\n"
|
2013-10-24 16:43:14 +02:00
|
|
|
"\treadreverse -- read N times in reverse order\n"
|
|
|
|
"\treadrandom -- read N times in random order\n"
|
|
|
|
"\treadmissing -- read N missing keys in random order\n"
|
|
|
|
"\treadwhilewriting -- 1 writer, N threads doing random "
|
|
|
|
"reads\n"
|
2015-03-18 21:50:52 +01:00
|
|
|
"\treadwhilemerging -- 1 merger, N threads doing random "
|
|
|
|
"reads\n"
|
2013-10-24 16:43:14 +02:00
|
|
|
"\treadrandomwriterandom -- N threads doing random-read, "
|
|
|
|
"random-write\n"
|
|
|
|
"\tprefixscanrandom -- prefix scan N times in random order\n"
|
|
|
|
"\tupdaterandom -- N threads doing read-modify-write for random "
|
|
|
|
"keys\n"
|
|
|
|
"\tappendrandom -- N threads doing read-modify-write with "
|
|
|
|
"growing values\n"
|
|
|
|
"\tmergerandom -- same as updaterandom/appendrandom using merge"
|
|
|
|
" operator. "
|
|
|
|
"Must be used with merge_operator\n"
|
2014-01-11 02:33:56 +01:00
|
|
|
"\treadrandommergerandom -- perform N random read-or-merge "
|
|
|
|
"operations. Must be used with merge_operator\n"
|
2014-02-03 22:13:36 +01:00
|
|
|
"\tnewiterator -- repeated iterator creation\n"
|
2015-03-30 20:28:25 +02:00
|
|
|
"\tseekrandom -- N random seeks, call Next seek_nexts times "
|
|
|
|
"per seek\n"
|
|
|
|
"\tseekrandomwhilewriting -- seekrandom and 1 thread doing "
|
|
|
|
"overwrite\n"
|
|
|
|
"\tseekrandomwhilemerging -- seekrandom and 1 thread doing "
|
|
|
|
"merge\n"
|
2013-10-24 16:43:14 +02:00
|
|
|
"\tcrc32c -- repeated crc32c of 4K of data\n"
|
2014-05-01 20:09:32 +02:00
|
|
|
"\txxhash -- repeated xxHash of 4K of data\n"
|
2013-10-24 16:43:14 +02:00
|
|
|
"\tacquireload -- load N*1000 times\n"
|
SkipListRep::LookaheadIterator
Summary:
This diff introduces the `lookahead` argument to `SkipListFactory()`. This is an
optimization for the tailing use case which includes many seeks. E.g. consider
the following operations on a skip list iterator:
Seek(x), Next(), Next(), Seek(x+2), Next(), Seek(x+3), Next(), Next(), ...
If `lookahead` is positive, `SkipListRep` will return an iterator which also
keeps track of the previously visited node. Seek() then first does a linear
search starting from that node (up to `lookahead` steps). As in the tailing
example above, this may require fewer than ~log(n) comparisons as with regular
skip list search.
Test Plan:
Added a new benchmark (`fillseekseq`) which simulates the usage pattern. It
first writes N records (with consecutive keys), then measures how much time it
takes to read them by calling `Seek()` and `Next()`.
$ time ./db_bench -num 10000000 -benchmarks fillseekseq -prefix_size 1 \
-key_size 8 -write_buffer_size $[1024*1024*1024] -value_size 50 \
-seekseq_next 2 -skip_list_lookahead=0
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.389 micros/op 2569047 ops/sec;
real 0m21.806s
user 0m12.106s
sys 0m9.672s
$ time ./db_bench [...] -skip_list_lookahead=2
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.153 micros/op 6540684 ops/sec;
real 0m19.469s
user 0m10.192s
sys 0m9.252s
Reviewers: ljin, sdong, igor
Reviewed By: igor
Subscribers: dhruba, leveldb, march, lovro
Differential Revision: https://reviews.facebook.net/D23997
2014-09-24 00:52:28 +02:00
|
|
|
"\tfillseekseq -- write N values in sequential key, then read "
|
|
|
|
"them by seeking to each key\n"
|
2015-05-29 23:36:35 +02:00
|
|
|
"\trandomtransaction -- execute N random transactions and "
|
|
|
|
"verify correctness\n"
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
"\trandomreplacekeys -- randomly replaces N keys by deleting "
|
|
|
|
"the old version and putting the new version\n\n"
|
2013-10-24 16:43:14 +02:00
|
|
|
"Meta operations:\n"
|
|
|
|
"\tcompact -- Compact the entire DB\n"
|
|
|
|
"\tstats -- Print DB stats\n"
|
|
|
|
"\tlevelstats -- Print the number of files and bytes per level\n"
|
|
|
|
"\tsstables -- Print sstable info\n"
|
|
|
|
"\theapprofile -- Dump a heap profile (if supported by this"
|
|
|
|
" port)\n");
|
|
|
|
|
|
|
|
DEFINE_int64(num, 1000000, "Number of key/values to place in database");
|
|
|
|
|
|
|
|
DEFINE_int64(numdistinct, 1000,
|
|
|
|
"Number of distinct keys to use. Used in RandomWithVerify to "
|
|
|
|
"read/write on fewer keys so that gets are more likely to find the"
|
|
|
|
" key and puts are more likely to update the same key");
|
|
|
|
|
2014-01-11 02:33:56 +01:00
|
|
|
DEFINE_int64(merge_keys, -1,
|
|
|
|
"Number of distinct keys to use for MergeRandom and "
|
|
|
|
"ReadRandomMergeRandom. "
|
|
|
|
"If negative, there will be FLAGS_num keys.");
|
2014-08-19 03:15:01 +02:00
|
|
|
DEFINE_int32(num_column_families, 1, "Number of Column Families to use.");
|
2014-01-11 02:33:56 +01:00
|
|
|
|
2014-12-23 03:39:28 +01:00
|
|
|
DEFINE_int32(
|
2014-12-24 17:54:28 +01:00
|
|
|
num_hot_column_families, 0,
|
2014-12-23 03:39:28 +01:00
|
|
|
"Number of Hot Column Families. If more than 0, only write to this "
|
|
|
|
"number of column families. After finishing all the writes to them, "
|
|
|
|
"create new set of column families and insert to them. Only used "
|
|
|
|
"when num_column_families > 1.");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int64(reads, -1, "Number of read operations to do. "
|
|
|
|
"If negative, do FLAGS_num reads.");
|
|
|
|
|
2014-03-28 17:21:20 +01:00
|
|
|
DEFINE_int32(bloom_locality, 0, "Control bloom filter probes locality");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int64(seed, 0, "Seed base for random number generators. "
|
|
|
|
"When 0 it is deterministic.");
|
|
|
|
|
|
|
|
DEFINE_int32(threads, 1, "Number of concurrent threads to run.");
|
|
|
|
|
|
|
|
DEFINE_int32(duration, 0, "Time in seconds for the random-ops tests to run."
|
|
|
|
" When 0 then num & reads determine the test duration");
|
|
|
|
|
|
|
|
DEFINE_int32(value_size, 100, "Size of each value");
|
2013-08-23 23:49:57 +02:00
|
|
|
|
2014-10-17 19:20:25 +02:00
|
|
|
DEFINE_int32(seek_nexts, 0,
|
|
|
|
"How many times to call Next() after Seek() in "
|
2015-03-30 20:28:25 +02:00
|
|
|
"fillseekseq, seekrandom, seekrandomwhilewriting and "
|
|
|
|
"seekrandomwhilemerging");
|
SkipListRep::LookaheadIterator
Summary:
This diff introduces the `lookahead` argument to `SkipListFactory()`. This is an
optimization for the tailing use case which includes many seeks. E.g. consider
the following operations on a skip list iterator:
Seek(x), Next(), Next(), Seek(x+2), Next(), Seek(x+3), Next(), Next(), ...
If `lookahead` is positive, `SkipListRep` will return an iterator which also
keeps track of the previously visited node. Seek() then first does a linear
search starting from that node (up to `lookahead` steps). As in the tailing
example above, this may require fewer than ~log(n) comparisons as with regular
skip list search.
Test Plan:
Added a new benchmark (`fillseekseq`) which simulates the usage pattern. It
first writes N records (with consecutive keys), then measures how much time it
takes to read them by calling `Seek()` and `Next()`.
$ time ./db_bench -num 10000000 -benchmarks fillseekseq -prefix_size 1 \
-key_size 8 -write_buffer_size $[1024*1024*1024] -value_size 50 \
-seekseq_next 2 -skip_list_lookahead=0
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.389 micros/op 2569047 ops/sec;
real 0m21.806s
user 0m12.106s
sys 0m9.672s
$ time ./db_bench [...] -skip_list_lookahead=2
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.153 micros/op 6540684 ops/sec;
real 0m19.469s
user 0m10.192s
sys 0m9.252s
Reviewers: ljin, sdong, igor
Reviewed By: igor
Subscribers: dhruba, leveldb, march, lovro
Differential Revision: https://reviews.facebook.net/D23997
2014-09-24 00:52:28 +02:00
|
|
|
|
2015-03-18 21:50:52 +01:00
|
|
|
DEFINE_bool(reverse_iterator, false,
|
|
|
|
"When true use Prev rather than Next for iterators that do "
|
|
|
|
"Seek and then Next");
|
|
|
|
|
2014-08-27 19:39:31 +02:00
|
|
|
DEFINE_bool(use_uint64_comparator, false, "use Uint64 user comparator");
|
2013-01-03 21:11:50 +01:00
|
|
|
|
2014-09-25 22:34:51 +02:00
|
|
|
DEFINE_int64(batch_size, 1, "Batch size");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
static bool ValidateKeySize(const char* flagname, int32_t value) {
|
|
|
|
return true;
|
|
|
|
}
|
2014-04-09 20:25:21 +02:00
|
|
|
|
2015-08-21 23:25:34 +02:00
|
|
|
static bool ValidateUint32Range(const char* flagname, uint64_t value) {
|
|
|
|
if (value > std::numeric_limits<uint32_t>::max()) {
|
Fixing race condition in DBTest.DynamicMemtableOptions
Summary:
This patch fixes a race condition in DBTEst.DynamicMemtableOptions. In rare cases,
it was possible that the main thread would fill up both memtables before the flush
job acquired its work. Then, the flush job was flushing both memtables together,
producing only one L0 file while the test expected two. Now, the test waits for
flushes to finish earlier, to make sure that the memtables are flushed in separate
flush jobs.
Test Plan:
Insert "usleep(10000);" after "IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);" in BGWorkFlush()
to make the issue more likely. Then test with:
make db_test && time while ./db_test --gtest_filter=*DynamicMemtableOptions; do true; done
Reviewers: rven, sdong, yhchiang, anthony, igor
Reviewed By: igor
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D45429
2015-08-25 02:04:18 +02:00
|
|
|
fprintf(stderr, "Invalid value for --%s: %lu, overflow\n", flagname,
|
2015-08-21 23:25:34 +02:00
|
|
|
(unsigned long)value);
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(key_size, 16, "size of each key");
|
2013-03-01 23:10:09 +01:00
|
|
|
|
2014-04-11 21:15:09 +02:00
|
|
|
DEFINE_int32(num_multi_db, 0,
|
|
|
|
"Number of DBs used in the benchmark. 0 means single DB.");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_double(compression_ratio, 0.5, "Arrange to generate values that shrink"
|
|
|
|
" to this fraction of their original size after compression");
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2015-03-07 01:38:50 +01:00
|
|
|
DEFINE_double(read_random_exp_range, 0.0,
|
|
|
|
"Read random's key will be generated using distribution of "
|
2015-07-15 19:21:09 +02:00
|
|
|
"num * exp(-r) where r is uniform number from 0 to this value. "
|
2015-03-07 01:38:50 +01:00
|
|
|
"The larger the number is, the more skewed the reads are. "
|
|
|
|
"Only used in readrandom and multireadrandom benchmarks.");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_bool(histogram, false, "Print histogram of operation timings");
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2014-07-07 19:53:31 +02:00
|
|
|
DEFINE_bool(enable_numa, false,
|
|
|
|
"Make operations aware of NUMA architecture and bind memory "
|
|
|
|
"and cpus corresponding to nodes together. In NUMA, memory "
|
|
|
|
"in same node as CPUs are closer when compared to memory in "
|
|
|
|
"other nodes. Reads can be faster when the process is bound to "
|
|
|
|
"CPU and memory of same node. Use \"$numactl --hardware\" command "
|
|
|
|
"to see NUMA memory architecture.");
|
|
|
|
|
2014-12-02 21:09:20 +01:00
|
|
|
DEFINE_int64(db_write_buffer_size, rocksdb::Options().db_write_buffer_size,
|
|
|
|
"Number of bytes to buffer in all memtables before compacting");
|
|
|
|
|
Add monitoring for universal compaction and add counters for compaction IO
Summary:
Adds these counters
{ WAL_FILE_SYNCED, "rocksdb.wal.synced" }
number of writes that request a WAL sync
{ WAL_FILE_BYTES, "rocksdb.wal.bytes" },
number of bytes written to the WAL
{ WRITE_DONE_BY_SELF, "rocksdb.write.self" },
number of writes processed by the calling thread
{ WRITE_DONE_BY_OTHER, "rocksdb.write.other" },
number of writes not processed by the calling thread. Instead these were
processed by the current holder of the write lock
{ WRITE_WITH_WAL, "rocksdb.write.wal" },
number of writes that request WAL logging
{ COMPACT_READ_BYTES, "rocksdb.compact.read.bytes" },
number of bytes read during compaction
{ COMPACT_WRITE_BYTES, "rocksdb.compact.write.bytes" },
number of bytes written during compaction
Per-interval stats output was updated with WAL stats and correct stats for universal compaction
including a correct value for write-amplification. It now looks like:
Compactions
Level Files Size(MB) Score Time(sec) Read(MB) Write(MB) Rn(MB) Rnp1(MB) Wnew(MB) RW-Amplify Read(MB/s) Write(MB/s) Rn Rnp1 Wnp1 NewW Count Ln-stall Stall-cnt
--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
0 7 464 46.4 281 3411 3875 3411 0 3875 2.1 12.1 13.8 621 0 240 240 628 0.0 0
Uptime(secs): 310.8 total, 2.0 interval
Writes cumulative: 9999999 total, 9999999 batches, 1.0 per batch, 1.22 ingest GB
WAL cumulative: 9999999 WAL writes, 9999999 WAL syncs, 1.00 writes per sync, 1.22 GB written
Compaction IO cumulative (GB): 1.22 new, 3.33 read, 3.78 write, 7.12 read+write
Compaction IO cumulative (MB/sec): 4.0 new, 11.0 read, 12.5 write, 23.4 read+write
Amplification cumulative: 4.1 write, 6.8 compaction
Writes interval: 100000 total, 100000 batches, 1.0 per batch, 12.5 ingest MB
WAL interval: 100000 WAL writes, 100000 WAL syncs, 1.00 writes per sync, 0.01 MB written
Compaction IO interval (MB): 12.49 new, 14.98 read, 21.50 write, 36.48 read+write
Compaction IO interval (MB/sec): 6.4 new, 7.6 read, 11.0 write, 18.6 read+write
Amplification interval: 101.7 write, 102.9 compaction
Stalls(secs): 142.924 level0_slowdown, 0.000 level0_numfiles, 0.805 memtable_compaction, 0.000 leveln_slowdown
Stalls(count): 132461 level0_slowdown, 0 level0_numfiles, 3 memtable_compaction, 0 leveln_slowdown
Task ID: #3329644, #3301695
Blame Rev:
Test Plan:
Revert Plan:
Database Impact:
Memcache Impact:
Other Notes:
EImportant:
- begin *PUBLIC* platform impact section -
Bugzilla: #
- end platform impact -
Reviewers: dhruba
CC: leveldb
Differential Revision: https://reviews.facebook.net/D14583
2013-12-09 22:43:34 +01:00
|
|
|
DEFINE_int64(write_buffer_size, rocksdb::Options().write_buffer_size,
|
2013-10-24 16:43:14 +02:00
|
|
|
"Number of bytes to buffer in memtable before compacting");
|
2011-04-12 21:38:58 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(max_write_buffer_number,
|
|
|
|
rocksdb::Options().max_write_buffer_number,
|
|
|
|
"The number of in-memory memtables. Each memtable is of size"
|
|
|
|
"write_buffer_size.");
|
2012-10-19 23:00:53 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(min_write_buffer_number_to_merge,
|
|
|
|
rocksdb::Options().min_write_buffer_number_to_merge,
|
|
|
|
"The minimum number of write buffers that will be merged together"
|
|
|
|
"before writing to storage. This is cheap because it is an"
|
|
|
|
"in-memory merge. If this feature is not enabled, then all these"
|
|
|
|
"write buffers are flushed to L0 as separate files and this "
|
|
|
|
"increases read amplification because a get request has to check"
|
|
|
|
" in all of these files. Also, an in-memory merge may result in"
|
|
|
|
" writing less data to storage if there are duplicate records "
|
|
|
|
" in each of these individual write buffers.");
|
2013-06-11 23:23:58 +02:00
|
|
|
|
Support saving history in memtable_list
Summary:
For transactions, we are using the memtables to validate that there are no write conflicts. But after flushing, we don't have any memtables, and transactions could fail to commit. So we want to someone keep around some extra history to use for conflict checking. In addition, we want to provide a way to increase the size of this history if too many transactions fail to commit.
After chatting with people, it seems like everyone prefers just using Memtables to store this history (instead of a separate history structure). It seems like the best place for this is abstracted inside the memtable_list. I decide to create a separate list in MemtableListVersion as using the same list complicated the flush/installalflushresults logic too much.
This diff adds a new parameter to control how much memtable history to keep around after flushing. However, it sounds like people aren't too fond of adding new parameters. So I am making the default size of flushed+not-flushed memtables be set to max_write_buffers. This should not change the maximum amount of memory used, but make it more likely we're using closer the the limit. (We are now postponing deleting flushed memtables until the max_write_buffer limit is reached). So while we might use more memory on average, we are still obeying the limit set (and you could argue it's better to go ahead and use up memory now instead of waiting for a write stall to happen to test this limit).
However, if people are opposed to this default behavior, we can easily set it to 0 and require this parameter be set in order to use transactions.
Test Plan: Added a xfunc test to play around with setting different values of this parameter in all tests. Added testing in memtablelist_test and planning on adding more testing here.
Reviewers: sdong, rven, igor
Reviewed By: igor
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D37443
2015-05-29 01:34:24 +02:00
|
|
|
DEFINE_int32(max_write_buffer_number_to_maintain,
|
|
|
|
rocksdb::Options().max_write_buffer_number_to_maintain,
|
|
|
|
"The total maximum number of write buffers to maintain in memory "
|
|
|
|
"including copies of buffers that have already been flushed. "
|
|
|
|
"Unlike max_write_buffer_number, this parameter does not affect "
|
|
|
|
"flushing. This controls the minimum amount of write history "
|
|
|
|
"that will be available in memory for conflict checking when "
|
|
|
|
"Transactions are used. If this value is too low, some "
|
|
|
|
"transactions may fail at commit time due to not being able to "
|
|
|
|
"determine whether there were any write conflicts. Setting this "
|
|
|
|
"value to 0 will cause write buffers to be freed immediately "
|
|
|
|
"after they are flushed. If this value is set to -1, "
|
|
|
|
"'max_write_buffer_number' will be used.");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(max_background_compactions,
|
|
|
|
rocksdb::Options().max_background_compactions,
|
|
|
|
"The maximum number of concurrent background compactions"
|
|
|
|
" that can occur in parallel.");
|
2012-10-19 23:00:53 +02:00
|
|
|
|
2015-08-21 23:25:34 +02:00
|
|
|
DEFINE_uint64(subcompactions, 1,
|
Fixing race condition in DBTest.DynamicMemtableOptions
Summary:
This patch fixes a race condition in DBTEst.DynamicMemtableOptions. In rare cases,
it was possible that the main thread would fill up both memtables before the flush
job acquired its work. Then, the flush job was flushing both memtables together,
producing only one L0 file while the test expected two. Now, the test waits for
flushes to finish earlier, to make sure that the memtables are flushed in separate
flush jobs.
Test Plan:
Insert "usleep(10000);" after "IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);" in BGWorkFlush()
to make the issue more likely. Then test with:
make db_test && time while ./db_test --gtest_filter=*DynamicMemtableOptions; do true; done
Reviewers: rven, sdong, yhchiang, anthony, igor
Reviewed By: igor
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D45429
2015-08-25 02:04:18 +02:00
|
|
|
"Maximum number of subcompactions to divide L0-L1 compactions "
|
|
|
|
"into.");
|
|
|
|
static const bool FLAGS_subcompactions_dummy
|
|
|
|
__attribute__((unused)) = RegisterFlagValidator(&FLAGS_subcompactions,
|
|
|
|
&ValidateUint32Range);
|
2015-08-21 23:25:34 +02:00
|
|
|
|
2014-03-05 19:27:17 +01:00
|
|
|
DEFINE_int32(max_background_flushes,
|
|
|
|
rocksdb::Options().max_background_flushes,
|
|
|
|
"The maximum number of concurrent background flushes"
|
|
|
|
" that can occur in parallel.");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
static rocksdb::CompactionStyle FLAGS_compaction_style_e;
|
|
|
|
DEFINE_int32(compaction_style, (int32_t) rocksdb::Options().compaction_style,
|
|
|
|
"style of compaction: level-based vs universal");
|
2013-06-14 07:09:08 +02:00
|
|
|
|
2015-09-22 02:16:31 +02:00
|
|
|
static rocksdb::CompactionPri FLAGS_compaction_pri_e;
|
2015-10-30 22:42:18 +01:00
|
|
|
DEFINE_int32(compaction_pri, (int32_t)rocksdb::Options().compaction_pri,
|
2015-09-22 02:16:31 +02:00
|
|
|
"priority of files to compaction: by size or by data age");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(universal_size_ratio, 0,
|
|
|
|
"Percentage flexibility while comparing file size"
|
|
|
|
" (for universal compaction only).");
|
2013-06-14 07:09:08 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(universal_min_merge_width, 0, "The minimum number of files in a"
|
|
|
|
" single compaction run (for universal compaction only).");
|
2013-09-10 01:06:10 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(universal_max_merge_width, 0, "The max number of files to compact"
|
|
|
|
" in universal style compaction");
|
2013-09-10 01:06:10 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(universal_max_size_amplification_percent, 0,
|
|
|
|
"The max size amplification for universal style compaction");
|
2013-06-14 07:09:08 +02:00
|
|
|
|
2013-12-03 21:32:07 +01:00
|
|
|
DEFINE_int32(universal_compression_size_percent, -1,
|
|
|
|
"The percentage of the database to compress for universal "
|
|
|
|
"compaction. -1 means compress everything.");
|
|
|
|
|
2015-07-20 20:52:39 +02:00
|
|
|
DEFINE_bool(universal_allow_trivial_move, false,
|
2015-08-11 20:46:15 +02:00
|
|
|
"Allow trivial move in universal compaction.");
|
2015-07-20 20:52:39 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int64(cache_size, -1, "Number of bytes to use as a cache of uncompressed"
|
|
|
|
"data. Negative means use default settings.");
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2015-07-09 22:36:16 +02:00
|
|
|
DEFINE_bool(cache_index_and_filter_blocks, false,
|
|
|
|
"Cache index/filter blocks in block cache.");
|
|
|
|
|
2014-11-11 22:47:22 +01:00
|
|
|
DEFINE_int32(block_size,
|
|
|
|
static_cast<int32_t>(rocksdb::BlockBasedTableOptions().block_size),
|
2013-10-24 16:43:14 +02:00
|
|
|
"Number of bytes in a block.");
|
2012-09-13 18:23:18 +02:00
|
|
|
|
2014-08-25 23:22:05 +02:00
|
|
|
DEFINE_int32(block_restart_interval,
|
|
|
|
rocksdb::BlockBasedTableOptions().block_restart_interval,
|
2014-07-21 21:01:40 +02:00
|
|
|
"Number of keys between restart points "
|
|
|
|
"for delta encoding of keys.");
|
|
|
|
|
2013-09-02 08:23:40 +02:00
|
|
|
DEFINE_int64(compressed_cache_size, -1,
|
|
|
|
"Number of bytes to use as a cache of compressed data.");
|
|
|
|
|
2015-06-23 19:25:45 +02:00
|
|
|
DEFINE_int64(row_cache_size, 0,
|
|
|
|
"Number of bytes to use as a cache of individual rows"
|
|
|
|
" (0 = disabled).");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(open_files, rocksdb::Options().max_open_files,
|
|
|
|
"Maximum number of files to keep open at the same time"
|
|
|
|
" (use default if == 0)");
|
2011-05-21 04:17:43 +02:00
|
|
|
|
2015-09-30 18:51:31 +02:00
|
|
|
DEFINE_int32(file_opening_threads, rocksdb::Options().max_file_opening_threads,
|
|
|
|
"If open_files is set to -1, this option set the number of "
|
|
|
|
"threads that will be used to open files during DB::Open()");
|
|
|
|
|
2015-08-27 00:25:59 +02:00
|
|
|
DEFINE_int32(new_table_reader_for_compaction_inputs, true,
|
|
|
|
"If true, uses a separate file handle for compaction inputs");
|
|
|
|
|
|
|
|
DEFINE_int32(compaction_readahead_size, 0, "Compaction readahead size");
|
|
|
|
|
2015-10-29 19:34:34 +01:00
|
|
|
DEFINE_int32(random_access_max_buffer_size, 1024 * 1024,
|
|
|
|
"Maximum windows randomaccess buffer size");
|
2015-10-27 22:44:16 +01:00
|
|
|
|
2015-11-16 21:56:21 +01:00
|
|
|
DEFINE_int32(writable_file_max_buffer_size, 1024 * 1024,
|
|
|
|
"Maximum write buffer for Writable File");
|
2015-10-30 06:10:25 +01:00
|
|
|
|
|
|
|
DEFINE_int32(skip_table_builder_flush, false, "Skip flushing block in "
|
|
|
|
"table builder ");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(bloom_bits, -1, "Bloom filter bits per key. Negative means"
|
|
|
|
" use default settings.");
|
2014-03-05 19:27:17 +01:00
|
|
|
DEFINE_int32(memtable_bloom_bits, 0, "Bloom filter bits per key for memtable. "
|
|
|
|
"Negative means no bloom filter.");
|
2012-04-17 17:36:46 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_bool(use_existing_db, false, "If true, do not destroy the existing"
|
|
|
|
" database. If you set this flag and also specify a benchmark that"
|
|
|
|
" wants a fresh database, that benchmark will fail.");
|
2011-05-21 04:17:43 +02:00
|
|
|
|
Add argument --show_table_properties to db_bench
Summary:
Add argument --show_table_properties to db_bench
-show_table_properties (If true, then per-level table properties will be
printed on every stats-interval when stats_interval is set and
stats_per_interval is on.) type: bool default: false
Test Plan:
./db_bench --show_table_properties=1 --stats_interval=100000 --stats_per_interval=1
./db_bench --show_table_properties=1 --stats_interval=100000 --stats_per_interval=1 --num_column_families=2
Sample Output:
Compaction Stats [column_family_name_000001]
Level Files Size(MB) Score Read(GB) Rn(GB) Rnp1(GB) Write(GB) Wnew(GB) Moved(GB) W-Amp Rd(MB/s) Wr(MB/s) Comp(sec) Comp(cnt) Avg(sec) Stall(cnt) KeyIn KeyDrop
---------------------------------------------------------------------------------------------------------------------------------------------------------------------
L0 3/0 5 0.8 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 86.3 0 17 0.021 0 0 0
L1 5/0 9 0.9 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0 0 0.000 0 0 0
L2 9/0 16 0.2 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0 0 0.000 0 0 0
Sum 17/0 31 0.0 0.0 0.0 0.0 0.0 0.0 0.0 1.0 0.0 86.3 0 17 0.021 0 0 0
Int 0/0 0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 1.0 0.0 83.9 0 2 0.022 0 0 0
Flush(GB): cumulative 0.030, interval 0.004
Stalls(count): 0 level0_slowdown, 0 level0_numfiles, 0 memtable_compaction, 0 leveln_slowdown_soft, 0 leveln_slowdown_hard
Level[0]: # data blocks=2571; # entries=84813; raw key size=2035512; raw average key size=24.000000; raw value size=8481300; raw average value size=100.000000; data block size=5690119; index block size=82415; filter block size=0; (estimated) table size=5772534; filter policy name=N/A;
Level[1]: # data blocks=4285; # entries=141355; raw key size=3392520; raw average key size=24.000000; raw value size=14135500; raw average value size=100.000000; data block size=9487353; index block size=137377; filter block size=0; (estimated) table size=9624730; filter policy name=N/A;
Level[2]: # data blocks=7713; # entries=254439; raw key size=6106536; raw average key size=24.000000; raw value size=25443900; raw average value size=100.000000; data block size=17077893; index block size=247269; filter block size=0; (estimated) table size=17325162; filter policy name=N/A;
Level[3]: # data blocks=0; # entries=0; raw key size=0; raw average key size=0.000000; raw value size=0; raw average value size=0.000000; data block size=0; index block size=0; filter block size=0; (estimated) table size=0; filter policy name=N/A;
Level[4]: # data blocks=0; # entries=0; raw key size=0; raw average key size=0.000000; raw value size=0; raw average value size=0.000000; data block size=0; index block size=0; filter block size=0; (estimated) table size=0; filter policy name=N/A;
Level[5]: # data blocks=0; # entries=0; raw key size=0; raw average key size=0.000000; raw value size=0; raw average value size=0.000000; data block size=0; index block size=0; filter block size=0; (estimated) table size=0; filter policy name=N/A;
Level[6]: # data blocks=0; # entries=0; raw key size=0; raw average key size=0.000000; raw value size=0; raw average value size=0.000000; data block size=0; index block size=0; filter block size=0; (estimated) table size=0; filter policy name=N/A;
Reviewers: anthony, IslamAbdelRahman, MarkCallaghan, sdong, igor
Reviewed By: igor
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D45651
2015-08-27 03:27:23 +02:00
|
|
|
DEFINE_bool(show_table_properties, false,
|
|
|
|
"If true, then per-level table"
|
|
|
|
" properties will be printed on every stats-interval when"
|
|
|
|
" stats_interval is set and stats_per_interval is on.");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_string(db, "", "Use the db with the following name.");
|
2011-06-22 04:36:45 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
static bool ValidateCacheNumshardbits(const char* flagname, int32_t value) {
|
|
|
|
if (value >= 20) {
|
|
|
|
fprintf(stderr, "Invalid value for --%s: %d, must be < 20\n",
|
|
|
|
flagname, value);
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
DEFINE_int32(cache_numshardbits, -1, "Number of shards for the block cache"
|
|
|
|
" is 2 ** cache_numshardbits. Negative means use default settings."
|
|
|
|
" This is applied only if FLAGS_cache_size is non-negative.");
|
2012-05-17 02:22:33 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_bool(verify_checksum, false, "Verify checksum for every block read"
|
|
|
|
" from storage");
|
2012-05-19 09:05:48 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_bool(statistics, false, "Database statistics");
|
2013-10-04 06:49:15 +02:00
|
|
|
static class std::shared_ptr<rocksdb::Statistics> dbstats;
|
2012-05-30 08:18:16 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int64(writes, -1, "Number of write operations to do. If negative, do"
|
|
|
|
" --num reads.");
|
2012-08-16 00:44:23 +02:00
|
|
|
|
2015-03-18 21:50:52 +01:00
|
|
|
DEFINE_int32(writes_per_second, 0, "Per-thread rate limit on writes and merges "
|
|
|
|
" per second. No limit when <= 0. Only for the readwhilewriting "
|
|
|
|
" and readwhilemerging tests.");
|
2012-08-27 21:10:26 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_bool(sync, false, "Sync all writes to disk");
|
2012-08-20 01:05:02 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_bool(disable_data_sync, false, "If true, do not wait until data is"
|
|
|
|
" synced to disk.");
|
2013-03-01 20:25:13 +01:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_bool(use_fsync, false, "If true, issue fsync instead of fdatasync");
|
2013-03-01 20:25:13 +01:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_bool(disable_wal, false, "If true, do not write WAL for write.");
|
2012-10-15 19:10:56 +02:00
|
|
|
|
2014-02-25 19:43:46 +01:00
|
|
|
DEFINE_string(wal_dir, "", "If not empty, use the given dir for WAL");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(num_levels, 7, "The total number of levels");
|
2012-08-16 00:44:23 +02:00
|
|
|
|
2014-09-22 20:15:03 +02:00
|
|
|
DEFINE_int64(target_file_size_base, 2 * 1048576, "Target file size at level-1");
|
2012-08-16 00:44:23 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(target_file_size_multiplier, 1,
|
|
|
|
"A multiplier to compute target level-N file size (N >= 2)");
|
2013-05-21 20:37:06 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_uint64(max_bytes_for_level_base, 10 * 1048576, "Max bytes for level-1");
|
2012-08-20 01:05:02 +02:00
|
|
|
|
options.level_compaction_dynamic_level_bytes to allow RocksDB to pick size bases of levels dynamically.
Summary:
When having fixed max_bytes_for_level_base, the ratio of size of largest level and the second one can range from 0 to the multiplier. This makes LSM tree frequently irregular and unpredictable. It can also cause poor space amplification in some cases.
In this improvement (proposed by Igor Kabiljo), we introduce a parameter option.level_compaction_use_dynamic_max_bytes. When turning it on, RocksDB is free to pick a level base in the range of (options.max_bytes_for_level_base/options.max_bytes_for_level_multiplier, options.max_bytes_for_level_base] so that real level ratios are close to options.max_bytes_for_level_multiplier.
Test Plan: New unit tests and pass tests suites including valgrind.
Reviewers: MarkCallaghan, rven, yhchiang, igor, ikabiljo
Reviewed By: ikabiljo
Subscribers: yoshinorim, ikabiljo, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D31437
2015-02-05 20:44:17 +01:00
|
|
|
DEFINE_bool(level_compaction_dynamic_level_bytes, false,
|
|
|
|
"Whether level size base is dynamic");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(max_bytes_for_level_multiplier, 10,
|
|
|
|
"A multiplier to compute max bytes for level-N (N >= 2)");
|
2012-08-20 01:05:02 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
static std::vector<int> FLAGS_max_bytes_for_level_multiplier_additional_v;
|
|
|
|
DEFINE_string(max_bytes_for_level_multiplier_additional, "",
|
|
|
|
"A vector that specifies additional fanout per level");
|
2012-10-19 23:00:53 +02:00
|
|
|
|
Don't artificially inflate L0 score
Summary:
This turns out to be pretty bad because if we prioritize L0->L1 then L1 can grow artificially large, which makes L0->L1 more and more expensive. For example:
256MB @ L0 + 256MB @ L1 --> 512MB @ L1
256MB @ L0 + 512MB @ L1 --> 768MB @ L1
256MB @ L0 + 768MB @ L1 --> 1GB @ L1
....
256MB @ L0 + 10GB @ L1 --> 10.2GB @ L1
At some point we need to start compacting L1->L2 to speed up L0->L1.
Test Plan:
The performance improvement is massive for heavy write workload. This is the benchmark I ran: https://phabricator.fb.com/P19842671. Before this change, the benchmark took 47 minutes to complete. After, the benchmark finished in 2minutes. You can see full results here: https://phabricator.fb.com/P19842674
Also, we ran this diff on MongoDB on RocksDB on one replicaset. Before the change, our initial sync was so slow that it couldn't keep up with primary writes. After the change, the import finished without any issues
Reviewers: dynamike, MarkCallaghan, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D38637
2015-05-21 20:40:48 +02:00
|
|
|
DEFINE_int32(level0_stop_writes_trigger,
|
|
|
|
rocksdb::Options().level0_stop_writes_trigger,
|
|
|
|
"Number of files in level-0"
|
2013-10-24 16:43:14 +02:00
|
|
|
" that will trigger put stop.");
|
2012-08-31 08:06:17 +02:00
|
|
|
|
Don't artificially inflate L0 score
Summary:
This turns out to be pretty bad because if we prioritize L0->L1 then L1 can grow artificially large, which makes L0->L1 more and more expensive. For example:
256MB @ L0 + 256MB @ L1 --> 512MB @ L1
256MB @ L0 + 512MB @ L1 --> 768MB @ L1
256MB @ L0 + 768MB @ L1 --> 1GB @ L1
....
256MB @ L0 + 10GB @ L1 --> 10.2GB @ L1
At some point we need to start compacting L1->L2 to speed up L0->L1.
Test Plan:
The performance improvement is massive for heavy write workload. This is the benchmark I ran: https://phabricator.fb.com/P19842671. Before this change, the benchmark took 47 minutes to complete. After, the benchmark finished in 2minutes. You can see full results here: https://phabricator.fb.com/P19842674
Also, we ran this diff on MongoDB on RocksDB on one replicaset. Before the change, our initial sync was so slow that it couldn't keep up with primary writes. After the change, the import finished without any issues
Reviewers: dynamike, MarkCallaghan, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D38637
2015-05-21 20:40:48 +02:00
|
|
|
DEFINE_int32(level0_slowdown_writes_trigger,
|
|
|
|
rocksdb::Options().level0_slowdown_writes_trigger,
|
|
|
|
"Number of files in level-0"
|
2013-10-24 16:43:14 +02:00
|
|
|
" that will slow down writes.");
|
2013-02-20 02:51:06 +01:00
|
|
|
|
Don't artificially inflate L0 score
Summary:
This turns out to be pretty bad because if we prioritize L0->L1 then L1 can grow artificially large, which makes L0->L1 more and more expensive. For example:
256MB @ L0 + 256MB @ L1 --> 512MB @ L1
256MB @ L0 + 512MB @ L1 --> 768MB @ L1
256MB @ L0 + 768MB @ L1 --> 1GB @ L1
....
256MB @ L0 + 10GB @ L1 --> 10.2GB @ L1
At some point we need to start compacting L1->L2 to speed up L0->L1.
Test Plan:
The performance improvement is massive for heavy write workload. This is the benchmark I ran: https://phabricator.fb.com/P19842671. Before this change, the benchmark took 47 minutes to complete. After, the benchmark finished in 2minutes. You can see full results here: https://phabricator.fb.com/P19842674
Also, we ran this diff on MongoDB on RocksDB on one replicaset. Before the change, our initial sync was so slow that it couldn't keep up with primary writes. After the change, the import finished without any issues
Reviewers: dynamike, MarkCallaghan, rven, yhchiang, sdong
Reviewed By: sdong
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D38637
2015-05-21 20:40:48 +02:00
|
|
|
DEFINE_int32(level0_file_num_compaction_trigger,
|
|
|
|
rocksdb::Options().level0_file_num_compaction_trigger,
|
|
|
|
"Number of files in level-0"
|
2013-10-24 16:43:14 +02:00
|
|
|
" when compactions start");
|
2012-09-17 22:35:57 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
static bool ValidateInt32Percent(const char* flagname, int32_t value) {
|
|
|
|
if (value <= 0 || value>=100) {
|
|
|
|
fprintf(stderr, "Invalid value for --%s: %d, 0< pct <100 \n",
|
|
|
|
flagname, value);
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
DEFINE_int32(readwritepercent, 90, "Ratio of reads to reads/writes (expressed"
|
|
|
|
" as percentage) for the ReadRandomWriteRandom workload. The "
|
|
|
|
"default value 90 means 90% operations out of all reads and writes"
|
|
|
|
" operations are reads. In other words, 9 gets for every 1 put.");
|
|
|
|
|
2014-01-11 02:33:56 +01:00
|
|
|
DEFINE_int32(mergereadpercent, 70, "Ratio of merges to merges&reads (expressed"
|
|
|
|
" as percentage) for the ReadRandomMergeRandom workload. The"
|
|
|
|
" default value 70 means 70% out of all read and merge operations"
|
|
|
|
" are merges. In other words, 7 merges for every 3 gets.");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(deletepercent, 2, "Percentage of deletes out of reads/writes/"
|
|
|
|
"deletes (used in RandomWithVerify only). RandomWithVerify "
|
|
|
|
"calculates writepercent as (100 - FLAGS_readwritepercent - "
|
|
|
|
"deletepercent), so deletepercent must be smaller than (100 - "
|
|
|
|
"FLAGS_readwritepercent)");
|
|
|
|
|
2014-12-22 12:04:45 +01:00
|
|
|
DEFINE_uint64(delete_obsolete_files_period_micros, 0,
|
|
|
|
"Ignored. Left here for backward compatibility");
|
2013-10-24 16:43:14 +02:00
|
|
|
|
2015-10-14 21:43:00 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
DEFINE_bool(optimistic_transaction_db, false,
|
2015-05-29 23:36:35 +02:00
|
|
|
"Open a OptimisticTransactionDB instance. "
|
|
|
|
"Required for randomtransaction benchmark.");
|
|
|
|
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
DEFINE_bool(transaction_db, false,
|
|
|
|
"Open a TransactionDB instance. "
|
|
|
|
"Required for randomtransaction benchmark.");
|
|
|
|
|
2015-05-29 23:36:35 +02:00
|
|
|
DEFINE_uint64(transaction_sets, 2,
|
|
|
|
"Number of keys each transaction will "
|
|
|
|
"modify (use in RandomTransaction only). Max: 9999");
|
|
|
|
|
2015-08-29 01:09:34 +02:00
|
|
|
DEFINE_bool(transaction_set_snapshot, false,
|
|
|
|
"Setting to true will have each transaction call SetSnapshot()"
|
|
|
|
" upon creation.");
|
|
|
|
|
2015-05-29 23:36:35 +02:00
|
|
|
DEFINE_int32(transaction_sleep, 0,
|
|
|
|
"Max microseconds to sleep in between "
|
|
|
|
"reading and writing a value (used in RandomTransaction only). ");
|
|
|
|
|
2015-08-29 01:09:34 +02:00
|
|
|
DEFINE_uint64(transaction_lock_timeout, 100,
|
|
|
|
"If using a transaction_db, specifies the lock wait timeout in"
|
|
|
|
" milliseconds before failing a transaction waiting on a lock");
|
2015-10-14 21:43:00 +02:00
|
|
|
#endif // ROCKSDB_LITE
|
2015-08-29 01:09:34 +02:00
|
|
|
|
Add options.compaction_measure_io_stats to print write I/O stats in compactions
Summary:
Add options.compaction_measure_io_stats to print out / pass to listener accumulated time spent on write calls. Example outputs in info logs:
2015/08/12-16:27:59.463944 7fd428bff700 (Original Log Time 2015/08/12-16:27:59.463922) EVENT_LOG_v1 {"time_micros": 1439422079463897, "job": 6, "event": "compaction_finished", "output_level": 1, "num_output_files": 4, "total_output_size": 6900525, "num_input_records": 111483, "num_output_records": 106877, "file_write_nanos": 15663206, "file_range_sync_nanos": 649588, "file_fsync_nanos": 349614797, "file_prepare_write_nanos": 1505812, "lsm_state": [2, 4, 0, 0, 0, 0, 0]}
Add two more counters in iostats_context.
Also add a parameter of db_bench.
Test Plan: Add a unit test. Also manually verify LOG outputs in db_bench
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D44115
2015-08-13 02:24:45 +02:00
|
|
|
DEFINE_bool(compaction_measure_io_stats, false,
|
|
|
|
"Measure times spents on I/Os while in compactions. ");
|
|
|
|
|
2014-04-10 06:17:14 +02:00
|
|
|
namespace {
|
2013-10-24 16:43:14 +02:00
|
|
|
enum rocksdb::CompressionType StringToCompressionType(const char* ctype) {
|
|
|
|
assert(ctype);
|
|
|
|
|
|
|
|
if (!strcasecmp(ctype, "none"))
|
|
|
|
return rocksdb::kNoCompression;
|
|
|
|
else if (!strcasecmp(ctype, "snappy"))
|
|
|
|
return rocksdb::kSnappyCompression;
|
|
|
|
else if (!strcasecmp(ctype, "zlib"))
|
|
|
|
return rocksdb::kZlibCompression;
|
|
|
|
else if (!strcasecmp(ctype, "bzip2"))
|
|
|
|
return rocksdb::kBZip2Compression;
|
2014-02-08 03:12:30 +01:00
|
|
|
else if (!strcasecmp(ctype, "lz4"))
|
|
|
|
return rocksdb::kLZ4Compression;
|
|
|
|
else if (!strcasecmp(ctype, "lz4hc"))
|
|
|
|
return rocksdb::kLZ4HCCompression;
|
2015-08-28 00:40:42 +02:00
|
|
|
else if (!strcasecmp(ctype, "zstd"))
|
|
|
|
return rocksdb::kZSTDNotFinalCompression;
|
2013-10-24 16:43:14 +02:00
|
|
|
|
|
|
|
fprintf(stdout, "Cannot parse compression type '%s'\n", ctype);
|
|
|
|
return rocksdb::kSnappyCompression; //default value
|
|
|
|
}
|
2014-12-23 03:39:28 +01:00
|
|
|
|
2014-12-31 03:33:35 +01:00
|
|
|
std::string ColumnFamilyName(size_t i) {
|
2014-12-23 03:39:28 +01:00
|
|
|
if (i == 0) {
|
|
|
|
return rocksdb::kDefaultColumnFamilyName;
|
|
|
|
} else {
|
|
|
|
char name[100];
|
2014-12-31 03:33:35 +01:00
|
|
|
snprintf(name, sizeof(name), "column_family_name_%06zu", i);
|
2014-12-23 03:39:28 +01:00
|
|
|
return std::string(name);
|
|
|
|
}
|
|
|
|
}
|
2014-04-10 06:17:14 +02:00
|
|
|
} // namespace
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_string(compression_type, "snappy",
|
|
|
|
"Algorithm to use to compress the database");
|
|
|
|
static enum rocksdb::CompressionType FLAGS_compression_type_e =
|
2013-10-04 06:49:15 +02:00
|
|
|
rocksdb::kSnappyCompression;
|
2012-09-14 20:21:54 +02:00
|
|
|
|
2013-12-03 21:32:07 +01:00
|
|
|
DEFINE_int32(compression_level, -1,
|
|
|
|
"Compression level. For zlib this should be -1 for the "
|
|
|
|
"default level, or between 0 and 9.");
|
|
|
|
|
|
|
|
static bool ValidateCompressionLevel(const char* flagname, int32_t value) {
|
|
|
|
if (value < -1 || value > 9) {
|
|
|
|
fprintf(stderr, "Invalid value for --%s: %d, must be between -1 and 9\n",
|
|
|
|
flagname, value);
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2014-01-17 21:46:06 +01:00
|
|
|
static const bool FLAGS_compression_level_dummy __attribute__((unused)) =
|
2014-05-09 02:25:13 +02:00
|
|
|
RegisterFlagValidator(&FLAGS_compression_level, &ValidateCompressionLevel);
|
2013-12-03 21:32:07 +01:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(min_level_to_compress, -1, "If non-negative, compression starts"
|
|
|
|
" from this level. Levels with number < min_level_to_compress are"
|
|
|
|
" not compressed. Otherwise, apply compression_type to "
|
|
|
|
"all levels.");
|
|
|
|
|
|
|
|
static bool ValidateTableCacheNumshardbits(const char* flagname,
|
|
|
|
int32_t value) {
|
|
|
|
if (0 >= value || value > 20) {
|
|
|
|
fprintf(stderr, "Invalid value for --%s: %d, must be 0 < val <= 20\n",
|
|
|
|
flagname, value);
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
DEFINE_int32(table_cache_numshardbits, 4, "");
|
2012-11-01 01:02:24 +01:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_string(hdfs, "", "Name of hdfs environment");
|
2012-06-08 10:11:14 +02:00
|
|
|
// posix or hdfs environment
|
2013-10-04 06:49:15 +02:00
|
|
|
static rocksdb::Env* FLAGS_env = rocksdb::Env::Default();
|
2012-06-08 10:11:14 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int64(stats_interval, 0, "Stats are reported every N operations when "
|
|
|
|
"this is greater than zero. When 0 the interval grows over time.");
|
2012-10-03 15:51:11 +02:00
|
|
|
|
2015-03-30 21:58:32 +02:00
|
|
|
DEFINE_int64(stats_interval_seconds, 0, "Report stats every N seconds. This "
|
|
|
|
"overrides stats_interval when both are > 0.");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(stats_per_interval, 0, "Reports additional stats per interval when"
|
|
|
|
" this is greater than 0.");
|
2012-10-26 22:37:21 +02:00
|
|
|
|
db_bench periodically writes QPS to CSV file
Summary:
This is part of an effort to better understand and optimize RocksDB stalls under high load. I added a feature to db_bench to periodically write QPS to CSV files. That way we can nicely see how our QPS changes in time (especially when DB is stalled) and can do a better job of evaluating our stall system (i.e. we want the QPS to be as constant as possible, as opposed to having bunch of stalls)
Cool part of CSV files is that we can easily graph them -- there are a bunch of tools available.
Test Plan:
Ran ./db_bench --report_interval_seconds=10 --benchmarks=fillrandom --num=10000000
and observed this in report.csv:
secs_elapsed,interval_qps
10,2725860
20,1980480
30,1863456
40,1454359
50,1460389
Reviewers: sdong, MarkCallaghan, rven, yhchiang
Reviewed By: yhchiang
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D40047
2015-06-12 23:31:53 +02:00
|
|
|
DEFINE_int64(report_interval_seconds, 0,
|
|
|
|
"If greater than zero, it will write simple stats in CVS format "
|
|
|
|
"to --report_file every N seconds");
|
|
|
|
|
|
|
|
DEFINE_string(report_file, "report.csv",
|
|
|
|
"Filename where some simple stats are reported to (if "
|
|
|
|
"--report_interval_seconds is bigger than 0)");
|
|
|
|
|
2015-03-06 20:22:06 +01:00
|
|
|
DEFINE_int32(thread_status_per_interval, 0,
|
|
|
|
"Takes and report a snapshot of the current status of each thread"
|
|
|
|
" when this is greater than 0.");
|
|
|
|
|
2013-11-18 20:32:54 +01:00
|
|
|
DEFINE_int32(perf_level, 0, "Level of perf collection");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
static bool ValidateRateLimit(const char* flagname, double value) {
|
2015-07-02 01:13:49 +02:00
|
|
|
const double EPSILON = 1e-10;
|
2013-10-24 16:43:14 +02:00
|
|
|
if ( value < -EPSILON ) {
|
|
|
|
fprintf(stderr, "Invalid value for --%s: %12.6f, must be >= 0.0\n",
|
|
|
|
flagname, value);
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
DEFINE_double(soft_rate_limit, 0.0, "");
|
2013-08-06 00:43:49 +02:00
|
|
|
|
2015-09-11 23:31:23 +02:00
|
|
|
DEFINE_double(hard_rate_limit, 0.0, "DEPRECATED");
|
|
|
|
|
2015-11-06 20:34:06 +01:00
|
|
|
DEFINE_uint64(hard_pending_compaction_bytes_limit, 128ull * 1024 * 1024 * 1024,
|
2015-09-11 23:31:23 +02:00
|
|
|
"Stop writes if pending compaction bytes exceed this number");
|
2012-10-26 22:37:21 +02:00
|
|
|
|
2015-05-16 00:52:51 +02:00
|
|
|
DEFINE_uint64(delayed_write_rate, 2097152u,
|
|
|
|
"Limited bytes allowed to DB when soft_rate_limit or "
|
|
|
|
"level0_slowdown_writes_trigger triggers");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(rate_limit_delay_max_milliseconds, 1000,
|
|
|
|
"When hard_rate_limit is set then this is the max time a put will"
|
|
|
|
" be stalled.");
|
2013-03-02 21:56:04 +01:00
|
|
|
|
2015-05-13 02:40:27 +02:00
|
|
|
DEFINE_uint64(rate_limiter_bytes_per_sec, 0, "Set options.rate_limiter value.");
|
|
|
|
|
2015-06-18 01:44:52 +02:00
|
|
|
DEFINE_uint64(
|
|
|
|
benchmark_write_rate_limit, 0,
|
|
|
|
"If non-zero, db_bench will rate-limit the writes going into RocksDB");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(max_grandparent_overlap_factor, 10, "Control maximum bytes of "
|
|
|
|
"overlaps in grandparent (i.e., level+2) before we stop building a"
|
|
|
|
" single file in a level->level+1 compaction.");
|
2012-11-14 18:10:46 +01:00
|
|
|
|
2015-10-14 21:43:00 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_bool(readonly, false, "Run read only benchmarks.");
|
2015-10-14 21:43:00 +02:00
|
|
|
#endif // ROCKSDB_LITE
|
2012-11-06 04:18:49 +01:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_bool(disable_auto_compactions, false, "Do not auto trigger compactions");
|
2012-11-21 00:45:41 +01:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_int32(source_compaction_factor, 1, "Cap the size of data in level-K for"
|
|
|
|
" a compaction run that compacts Level-K with Level-(K+1) (for"
|
|
|
|
" K >= 1)");
|
2012-11-21 08:07:41 +01:00
|
|
|
|
2013-11-07 03:46:28 +01:00
|
|
|
DEFINE_uint64(wal_ttl_seconds, 0, "Set the TTL for the WAL Files in seconds.");
|
|
|
|
DEFINE_uint64(wal_size_limit_MB, 0, "Set the size limit for the WAL Files"
|
|
|
|
" in MB.");
|
2014-12-23 03:39:28 +01:00
|
|
|
DEFINE_uint64(max_total_wal_size, 0, "Set total max WAL size");
|
2012-11-26 22:56:45 +01:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_bool(bufferedio, rocksdb::EnvOptions().use_os_buffer,
|
|
|
|
"Allow buffered io using OS buffers");
|
2013-03-15 01:00:04 +01:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_bool(mmap_read, rocksdb::EnvOptions().use_mmap_reads,
|
|
|
|
"Allow reads to occur via mmap-ing files");
|
2013-03-15 01:00:04 +01:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_bool(mmap_write, rocksdb::EnvOptions().use_mmap_writes,
|
|
|
|
"Allow writes to occur via mmap-ing files");
|
2013-03-15 01:00:04 +01:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_bool(advise_random_on_open, rocksdb::Options().advise_random_on_open,
|
|
|
|
"Advise random access on table file open");
|
2013-05-18 00:53:01 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_string(compaction_fadvice, "NORMAL",
|
|
|
|
"Access pattern advice when a file is compacted");
|
|
|
|
static auto FLAGS_compaction_fadvice_e =
|
2013-10-04 06:49:15 +02:00
|
|
|
rocksdb::Options().access_hint_on_compaction_start;
|
2013-05-18 00:53:01 +02:00
|
|
|
|
2015-03-30 18:51:11 +02:00
|
|
|
DEFINE_bool(disable_flashcache_for_background_threads, false,
|
|
|
|
"Disable flashcache for background threads");
|
|
|
|
|
|
|
|
DEFINE_string(flashcache_dev, "", "Path to flashcache device");
|
|
|
|
|
2014-02-07 18:47:47 +01:00
|
|
|
DEFINE_bool(use_tailing_iterator, false,
|
|
|
|
"Use tailing iterator to access a series of keys instead of get");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_bool(use_adaptive_mutex, rocksdb::Options().use_adaptive_mutex,
|
|
|
|
"Use adaptive mutex");
|
|
|
|
|
|
|
|
DEFINE_uint64(bytes_per_sync, rocksdb::Options().bytes_per_sync,
|
2015-05-20 01:19:30 +02:00
|
|
|
"Allows OS to incrementally sync SST files to disk while they are"
|
2013-10-24 16:43:14 +02:00
|
|
|
" being written, in the background. Issue one request for every"
|
|
|
|
" bytes_per_sync written. 0 turns it off.");
|
2015-05-20 01:19:30 +02:00
|
|
|
|
|
|
|
DEFINE_uint64(wal_bytes_per_sync, rocksdb::Options().wal_bytes_per_sync,
|
|
|
|
"Allows OS to incrementally sync WAL files to disk while they are"
|
|
|
|
" being written, in the background. Issue one request for every"
|
|
|
|
" wal_bytes_per_sync written. 0 turns it off.");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_bool(filter_deletes, false, " On true, deletes use bloom-filter and drop"
|
|
|
|
" the delete if key not present");
|
|
|
|
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
DEFINE_bool(use_single_deletes, true,
|
|
|
|
"Use single deletes (used in RandomReplaceKeys only).");
|
|
|
|
|
|
|
|
DEFINE_double(stddev, 2000.0,
|
|
|
|
"Standard deviation of normal distribution used for picking keys"
|
|
|
|
" (used in RandomReplaceKeys only).");
|
|
|
|
|
2014-01-11 02:33:56 +01:00
|
|
|
DEFINE_int32(max_successive_merges, 0, "Maximum number of successive merge"
|
|
|
|
" operations on a key in the memtable");
|
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
static bool ValidatePrefixSize(const char* flagname, int32_t value) {
|
|
|
|
if (value < 0 || value>=2000000000) {
|
|
|
|
fprintf(stderr, "Invalid value for --%s: %d. 0<= PrefixSize <=2000000000\n",
|
|
|
|
flagname, value);
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
2014-02-12 19:41:55 +01:00
|
|
|
DEFINE_int32(prefix_size, 0, "control the prefix size for HashSkipList and "
|
|
|
|
"plain table");
|
2014-03-05 02:08:05 +01:00
|
|
|
DEFINE_int64(keys_per_prefix, 0, "control average number of keys generated "
|
|
|
|
"per prefix, 0 means no special handling of the prefix, "
|
|
|
|
"i.e. use the prefix comes with the generated random number.");
|
2014-08-14 05:49:58 +02:00
|
|
|
DEFINE_bool(enable_io_prio, false, "Lower the background flush/compaction "
|
|
|
|
"threads' IO priority");
|
CuckooTable: add one option to allow identity function for the first hash function
Summary:
MurmurHash becomes expensive when we do millions Get() a second in one
thread. Add this option to allow the first hash function to use identity
function as hash function. It results in QPS increase from 3.7M/s to
~4.3M/s. I did not observe improvement for end to end RocksDB
performance. This may be caused by other bottlenecks that I will address
in a separate diff.
Test Plan:
```
[ljin@dev1964 rocksdb] ./cuckoo_table_reader_test --enable_perf --file_dir=/dev/shm --write --identity_as_first_hash=0
==== Test CuckooReaderTest.WhenKeyExists
==== Test CuckooReaderTest.WhenKeyExistsWithUint64Comparator
==== Test CuckooReaderTest.CheckIterator
==== Test CuckooReaderTest.CheckIteratorUint64
==== Test CuckooReaderTest.WhenKeyNotFound
==== Test CuckooReaderTest.TestReadPerformance
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.272us (3.7 Mqps) with batch size of 0, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.138us (7.2 Mqps) with batch size of 10, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.142us (7.1 Mqps) with batch size of 25, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.142us (7.0 Mqps) with batch size of 50, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.144us (6.9 Mqps) with batch size of 100, # of found keys 125829120
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.201us (5.0 Mqps) with batch size of 0, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.121us (8.3 Mqps) with batch size of 10, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.123us (8.1 Mqps) with batch size of 25, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.121us (8.3 Mqps) with batch size of 50, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.112us (8.9 Mqps) with batch size of 100, # of found keys 104857600
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.251us (4.0 Mqps) with batch size of 0, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.107us (9.4 Mqps) with batch size of 10, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.099us (10.1 Mqps) with batch size of 25, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.100us (10.0 Mqps) with batch size of 50, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.116us (8.6 Mqps) with batch size of 100, # of found keys 83886080
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.189us (5.3 Mqps) with batch size of 0, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.095us (10.5 Mqps) with batch size of 10, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.096us (10.4 Mqps) with batch size of 25, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.098us (10.2 Mqps) with batch size of 50, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.105us (9.5 Mqps) with batch size of 100, # of found keys 73400320
[ljin@dev1964 rocksdb] ./cuckoo_table_reader_test --enable_perf --file_dir=/dev/shm --write --identity_as_first_hash=1
==== Test CuckooReaderTest.WhenKeyExists
==== Test CuckooReaderTest.WhenKeyExistsWithUint64Comparator
==== Test CuckooReaderTest.CheckIterator
==== Test CuckooReaderTest.CheckIteratorUint64
==== Test CuckooReaderTest.WhenKeyNotFound
==== Test CuckooReaderTest.TestReadPerformance
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.230us (4.3 Mqps) with batch size of 0, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.086us (11.7 Mqps) with batch size of 10, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.088us (11.3 Mqps) with batch size of 25, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.083us (12.1 Mqps) with batch size of 50, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.083us (12.1 Mqps) with batch size of 100, # of found keys 125829120
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.159us (6.3 Mqps) with batch size of 0, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 10, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.080us (12.6 Mqps) with batch size of 25, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.080us (12.5 Mqps) with batch size of 50, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.082us (12.2 Mqps) with batch size of 100, # of found keys 104857600
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.154us (6.5 Mqps) with batch size of 0, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.077us (13.0 Mqps) with batch size of 10, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.077us (12.9 Mqps) with batch size of 25, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 50, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.079us (12.6 Mqps) with batch size of 100, # of found keys 83886080
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.218us (4.6 Mqps) with batch size of 0, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.083us (12.0 Mqps) with batch size of 10, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.085us (11.7 Mqps) with batch size of 25, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.086us (11.6 Mqps) with batch size of 50, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 100, # of found keys 73400320
```
Reviewers: sdong, igor, yhchiang
Reviewed By: igor
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D23451
2014-09-18 20:00:48 +02:00
|
|
|
DEFINE_bool(identity_as_first_hash, false, "the first hash function of cuckoo "
|
|
|
|
"table becomes an identity function. This is only valid when key "
|
|
|
|
"is 8 bytes");
|
2013-08-23 08:10:02 +02:00
|
|
|
|
2015-03-18 21:50:52 +01:00
|
|
|
enum PutOrMerge {
|
|
|
|
kPut,
|
|
|
|
kMerge
|
|
|
|
};
|
|
|
|
|
2013-08-23 08:10:02 +02:00
|
|
|
enum RepFactory {
|
|
|
|
kSkipList,
|
|
|
|
kPrefixHash,
|
2014-03-05 19:27:17 +01:00
|
|
|
kVectorRep,
|
Add a new mem-table representation based on cuckoo hash.
Summary:
= Major Changes =
* Add a new mem-table representation, HashCuckooRep, which is based cuckoo hash.
Cuckoo hash uses multiple hash functions. This allows each key to have multiple
possible locations in the mem-table.
- Put: When insert a key, it will try to find whether one of its possible
locations is vacant and store the key. If none of its possible
locations are available, then it will kick out a victim key and
store at that location. The kicked-out victim key will then be
stored at a vacant space of its possible locations or kick-out
another victim. In this diff, the kick-out path (known as
cuckoo-path) is found using BFS, which guarantees to be the shortest.
- Get: Simply tries all possible locations of a key --- this guarantees
worst-case constant time complexity.
- Time complexity: O(1) for Get, and average O(1) for Put if the
fullness of the mem-table is below 80%.
- Default using two hash functions, the number of hash functions used
by the cuckoo-hash may dynamically increase if it fails to find a
short-enough kick-out path.
- Currently, HashCuckooRep does not support iteration and snapshots,
as our current main purpose of this is to optimize point access.
= Minor Changes =
* Add IsSnapshotSupported() to DB to indicate whether the current DB
supports snapshots. If it returns false, then DB::GetSnapshot() will
always return nullptr.
Test Plan:
Run existing tests. Will develop a test specifically for cuckoo hash in
the next diff.
Reviewers: sdong, haobo
Reviewed By: sdong
CC: leveldb, dhruba, igor
Differential Revision: https://reviews.facebook.net/D16155
2014-04-30 02:13:46 +02:00
|
|
|
kHashLinkedList,
|
|
|
|
kCuckoo
|
2013-08-23 08:10:02 +02:00
|
|
|
};
|
2014-04-10 06:17:14 +02:00
|
|
|
|
|
|
|
namespace {
|
2013-10-24 16:43:14 +02:00
|
|
|
enum RepFactory StringToRepFactory(const char* ctype) {
|
|
|
|
assert(ctype);
|
|
|
|
|
|
|
|
if (!strcasecmp(ctype, "skip_list"))
|
|
|
|
return kSkipList;
|
|
|
|
else if (!strcasecmp(ctype, "prefix_hash"))
|
|
|
|
return kPrefixHash;
|
|
|
|
else if (!strcasecmp(ctype, "vector"))
|
|
|
|
return kVectorRep;
|
2014-03-05 19:27:17 +01:00
|
|
|
else if (!strcasecmp(ctype, "hash_linkedlist"))
|
|
|
|
return kHashLinkedList;
|
Add a new mem-table representation based on cuckoo hash.
Summary:
= Major Changes =
* Add a new mem-table representation, HashCuckooRep, which is based cuckoo hash.
Cuckoo hash uses multiple hash functions. This allows each key to have multiple
possible locations in the mem-table.
- Put: When insert a key, it will try to find whether one of its possible
locations is vacant and store the key. If none of its possible
locations are available, then it will kick out a victim key and
store at that location. The kicked-out victim key will then be
stored at a vacant space of its possible locations or kick-out
another victim. In this diff, the kick-out path (known as
cuckoo-path) is found using BFS, which guarantees to be the shortest.
- Get: Simply tries all possible locations of a key --- this guarantees
worst-case constant time complexity.
- Time complexity: O(1) for Get, and average O(1) for Put if the
fullness of the mem-table is below 80%.
- Default using two hash functions, the number of hash functions used
by the cuckoo-hash may dynamically increase if it fails to find a
short-enough kick-out path.
- Currently, HashCuckooRep does not support iteration and snapshots,
as our current main purpose of this is to optimize point access.
= Minor Changes =
* Add IsSnapshotSupported() to DB to indicate whether the current DB
supports snapshots. If it returns false, then DB::GetSnapshot() will
always return nullptr.
Test Plan:
Run existing tests. Will develop a test specifically for cuckoo hash in
the next diff.
Reviewers: sdong, haobo
Reviewed By: sdong
CC: leveldb, dhruba, igor
Differential Revision: https://reviews.facebook.net/D16155
2014-04-30 02:13:46 +02:00
|
|
|
else if (!strcasecmp(ctype, "cuckoo"))
|
|
|
|
return kCuckoo;
|
2013-10-24 16:43:14 +02:00
|
|
|
|
|
|
|
fprintf(stdout, "Cannot parse memreptable %s\n", ctype);
|
|
|
|
return kSkipList;
|
|
|
|
}
|
2014-04-10 06:17:14 +02:00
|
|
|
} // namespace
|
|
|
|
|
2013-08-23 08:10:02 +02:00
|
|
|
static enum RepFactory FLAGS_rep_factory;
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_string(memtablerep, "skip_list", "");
|
2014-03-05 19:27:17 +01:00
|
|
|
DEFINE_int64(hash_bucket_count, 1024 * 1024, "hash bucket count");
|
2014-02-12 19:41:55 +01:00
|
|
|
DEFINE_bool(use_plain_table, false, "if use plain table "
|
|
|
|
"instead of block-based table format");
|
2014-08-18 20:59:38 +02:00
|
|
|
DEFINE_bool(use_cuckoo_table, false, "if use cuckoo table format");
|
|
|
|
DEFINE_double(cuckoo_hash_ratio, 0.9, "Hash ratio for Cuckoo SST table.");
|
2014-07-17 02:32:30 +02:00
|
|
|
DEFINE_bool(use_hash_search, false, "if use kHashSearch "
|
|
|
|
"instead of kBinarySearch. "
|
|
|
|
"This is valid if only we use BlockTable");
|
2014-09-08 19:37:05 +02:00
|
|
|
DEFINE_bool(use_block_based_filter, false, "if use kBlockBasedFilter "
|
|
|
|
"instead of kFullFilter for filter block. "
|
|
|
|
"This is valid if only we use BlockTable");
|
2013-10-24 16:43:14 +02:00
|
|
|
DEFINE_string(merge_operator, "", "The merge operator to use with the database."
|
|
|
|
"If a new merge operator is specified, be sure to use fresh"
|
|
|
|
" database The possible merge operators are defined in"
|
|
|
|
" utilities/merge_operators.h");
|
SkipListRep::LookaheadIterator
Summary:
This diff introduces the `lookahead` argument to `SkipListFactory()`. This is an
optimization for the tailing use case which includes many seeks. E.g. consider
the following operations on a skip list iterator:
Seek(x), Next(), Next(), Seek(x+2), Next(), Seek(x+3), Next(), Next(), ...
If `lookahead` is positive, `SkipListRep` will return an iterator which also
keeps track of the previously visited node. Seek() then first does a linear
search starting from that node (up to `lookahead` steps). As in the tailing
example above, this may require fewer than ~log(n) comparisons as with regular
skip list search.
Test Plan:
Added a new benchmark (`fillseekseq`) which simulates the usage pattern. It
first writes N records (with consecutive keys), then measures how much time it
takes to read them by calling `Seek()` and `Next()`.
$ time ./db_bench -num 10000000 -benchmarks fillseekseq -prefix_size 1 \
-key_size 8 -write_buffer_size $[1024*1024*1024] -value_size 50 \
-seekseq_next 2 -skip_list_lookahead=0
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.389 micros/op 2569047 ops/sec;
real 0m21.806s
user 0m12.106s
sys 0m9.672s
$ time ./db_bench [...] -skip_list_lookahead=2
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.153 micros/op 6540684 ops/sec;
real 0m19.469s
user 0m10.192s
sys 0m9.252s
Reviewers: ljin, sdong, igor
Reviewed By: igor
Subscribers: dhruba, leveldb, march, lovro
Differential Revision: https://reviews.facebook.net/D23997
2014-09-24 00:52:28 +02:00
|
|
|
DEFINE_int32(skip_list_lookahead, 0, "Used with skip_list memtablerep; try "
|
|
|
|
"linear search first for this many steps from the previous "
|
|
|
|
"position");
|
2014-10-29 22:24:34 +01:00
|
|
|
DEFINE_bool(report_file_operations, false, "if report number of file "
|
|
|
|
"operations");
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
|
2013-11-17 08:44:39 +01:00
|
|
|
static const bool FLAGS_soft_rate_limit_dummy __attribute__((unused)) =
|
2014-05-09 02:25:13 +02:00
|
|
|
RegisterFlagValidator(&FLAGS_soft_rate_limit, &ValidateRateLimit);
|
2013-11-17 08:44:39 +01:00
|
|
|
|
|
|
|
static const bool FLAGS_hard_rate_limit_dummy __attribute__((unused)) =
|
2014-05-09 02:25:13 +02:00
|
|
|
RegisterFlagValidator(&FLAGS_hard_rate_limit, &ValidateRateLimit);
|
2013-11-17 08:44:39 +01:00
|
|
|
|
|
|
|
static const bool FLAGS_prefix_size_dummy __attribute__((unused)) =
|
2014-05-09 02:25:13 +02:00
|
|
|
RegisterFlagValidator(&FLAGS_prefix_size, &ValidatePrefixSize);
|
2013-11-17 08:44:39 +01:00
|
|
|
|
|
|
|
static const bool FLAGS_key_size_dummy __attribute__((unused)) =
|
2014-05-09 02:25:13 +02:00
|
|
|
RegisterFlagValidator(&FLAGS_key_size, &ValidateKeySize);
|
2013-11-17 08:44:39 +01:00
|
|
|
|
|
|
|
static const bool FLAGS_cache_numshardbits_dummy __attribute__((unused)) =
|
2014-05-09 02:25:13 +02:00
|
|
|
RegisterFlagValidator(&FLAGS_cache_numshardbits,
|
|
|
|
&ValidateCacheNumshardbits);
|
2013-11-17 08:44:39 +01:00
|
|
|
|
|
|
|
static const bool FLAGS_readwritepercent_dummy __attribute__((unused)) =
|
2014-05-09 02:25:13 +02:00
|
|
|
RegisterFlagValidator(&FLAGS_readwritepercent, &ValidateInt32Percent);
|
2013-11-17 08:44:39 +01:00
|
|
|
|
2014-06-20 10:23:02 +02:00
|
|
|
DEFINE_int32(disable_seek_compaction, false,
|
|
|
|
"Not used, left here for backwards compatibility");
|
|
|
|
|
2013-11-17 08:44:39 +01:00
|
|
|
static const bool FLAGS_deletepercent_dummy __attribute__((unused)) =
|
2014-05-09 02:25:13 +02:00
|
|
|
RegisterFlagValidator(&FLAGS_deletepercent, &ValidateInt32Percent);
|
|
|
|
static const bool FLAGS_table_cache_numshardbits_dummy __attribute__((unused)) =
|
|
|
|
RegisterFlagValidator(&FLAGS_table_cache_numshardbits,
|
|
|
|
&ValidateTableCacheNumshardbits);
|
2013-11-17 08:44:39 +01:00
|
|
|
|
2013-10-04 06:49:15 +02:00
|
|
|
namespace rocksdb {
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2014-10-29 22:24:34 +01:00
|
|
|
namespace {
|
|
|
|
struct ReportFileOpCounters {
|
|
|
|
std::atomic<int> open_counter_;
|
|
|
|
std::atomic<int> read_counter_;
|
|
|
|
std::atomic<int> append_counter_;
|
|
|
|
std::atomic<uint64_t> bytes_read_;
|
|
|
|
std::atomic<uint64_t> bytes_written_;
|
|
|
|
};
|
|
|
|
|
|
|
|
// A special Env to records and report file operations in db_bench
|
|
|
|
class ReportFileOpEnv : public EnvWrapper {
|
|
|
|
public:
|
|
|
|
explicit ReportFileOpEnv(Env* base) : EnvWrapper(base) { reset(); }
|
|
|
|
|
|
|
|
void reset() {
|
|
|
|
counters_.open_counter_ = 0;
|
|
|
|
counters_.read_counter_ = 0;
|
|
|
|
counters_.append_counter_ = 0;
|
|
|
|
counters_.bytes_read_ = 0;
|
|
|
|
counters_.bytes_written_ = 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
Status NewSequentialFile(const std::string& f, unique_ptr<SequentialFile>* r,
|
2015-02-26 20:28:41 +01:00
|
|
|
const EnvOptions& soptions) override {
|
2014-10-29 22:24:34 +01:00
|
|
|
class CountingFile : public SequentialFile {
|
|
|
|
private:
|
|
|
|
unique_ptr<SequentialFile> target_;
|
|
|
|
ReportFileOpCounters* counters_;
|
|
|
|
|
|
|
|
public:
|
|
|
|
CountingFile(unique_ptr<SequentialFile>&& target,
|
|
|
|
ReportFileOpCounters* counters)
|
|
|
|
: target_(std::move(target)), counters_(counters) {}
|
|
|
|
|
2015-02-26 20:28:41 +01:00
|
|
|
virtual Status Read(size_t n, Slice* result, char* scratch) override {
|
2014-10-29 22:24:34 +01:00
|
|
|
counters_->read_counter_.fetch_add(1, std::memory_order_relaxed);
|
|
|
|
Status rv = target_->Read(n, result, scratch);
|
|
|
|
counters_->bytes_read_.fetch_add(result->size(),
|
|
|
|
std::memory_order_relaxed);
|
|
|
|
return rv;
|
|
|
|
}
|
|
|
|
|
2015-02-26 20:28:41 +01:00
|
|
|
virtual Status Skip(uint64_t n) override { return target_->Skip(n); }
|
2014-10-29 22:24:34 +01:00
|
|
|
};
|
|
|
|
|
|
|
|
Status s = target()->NewSequentialFile(f, r, soptions);
|
|
|
|
if (s.ok()) {
|
|
|
|
counters()->open_counter_.fetch_add(1, std::memory_order_relaxed);
|
|
|
|
r->reset(new CountingFile(std::move(*r), counters()));
|
|
|
|
}
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
Status NewRandomAccessFile(const std::string& f,
|
|
|
|
unique_ptr<RandomAccessFile>* r,
|
2015-02-26 20:28:41 +01:00
|
|
|
const EnvOptions& soptions) override {
|
2014-10-29 22:24:34 +01:00
|
|
|
class CountingFile : public RandomAccessFile {
|
|
|
|
private:
|
|
|
|
unique_ptr<RandomAccessFile> target_;
|
|
|
|
ReportFileOpCounters* counters_;
|
|
|
|
|
|
|
|
public:
|
|
|
|
CountingFile(unique_ptr<RandomAccessFile>&& target,
|
|
|
|
ReportFileOpCounters* counters)
|
|
|
|
: target_(std::move(target)), counters_(counters) {}
|
|
|
|
virtual Status Read(uint64_t offset, size_t n, Slice* result,
|
2015-02-26 20:28:41 +01:00
|
|
|
char* scratch) const override {
|
2014-10-29 22:24:34 +01:00
|
|
|
counters_->read_counter_.fetch_add(1, std::memory_order_relaxed);
|
|
|
|
Status rv = target_->Read(offset, n, result, scratch);
|
|
|
|
counters_->bytes_read_.fetch_add(result->size(),
|
|
|
|
std::memory_order_relaxed);
|
|
|
|
return rv;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
Status s = target()->NewRandomAccessFile(f, r, soptions);
|
|
|
|
if (s.ok()) {
|
|
|
|
counters()->open_counter_.fetch_add(1, std::memory_order_relaxed);
|
|
|
|
r->reset(new CountingFile(std::move(*r), counters()));
|
|
|
|
}
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
Status NewWritableFile(const std::string& f, unique_ptr<WritableFile>* r,
|
2015-02-26 20:28:41 +01:00
|
|
|
const EnvOptions& soptions) override {
|
2014-10-29 22:24:34 +01:00
|
|
|
class CountingFile : public WritableFile {
|
|
|
|
private:
|
|
|
|
unique_ptr<WritableFile> target_;
|
|
|
|
ReportFileOpCounters* counters_;
|
|
|
|
|
|
|
|
public:
|
|
|
|
CountingFile(unique_ptr<WritableFile>&& target,
|
|
|
|
ReportFileOpCounters* counters)
|
|
|
|
: target_(std::move(target)), counters_(counters) {}
|
|
|
|
|
2015-02-26 20:28:41 +01:00
|
|
|
Status Append(const Slice& data) override {
|
2014-10-29 22:24:34 +01:00
|
|
|
counters_->append_counter_.fetch_add(1, std::memory_order_relaxed);
|
|
|
|
Status rv = target_->Append(data);
|
|
|
|
counters_->bytes_written_.fetch_add(data.size(),
|
|
|
|
std::memory_order_relaxed);
|
|
|
|
return rv;
|
|
|
|
}
|
|
|
|
|
2015-09-11 18:57:02 +02:00
|
|
|
Status Truncate(uint64_t size) override { return target_->Truncate(size); }
|
2015-02-26 20:28:41 +01:00
|
|
|
Status Close() override { return target_->Close(); }
|
|
|
|
Status Flush() override { return target_->Flush(); }
|
|
|
|
Status Sync() override { return target_->Sync(); }
|
2014-10-29 22:24:34 +01:00
|
|
|
};
|
|
|
|
|
|
|
|
Status s = target()->NewWritableFile(f, r, soptions);
|
|
|
|
if (s.ok()) {
|
|
|
|
counters()->open_counter_.fetch_add(1, std::memory_order_relaxed);
|
|
|
|
r->reset(new CountingFile(std::move(*r), counters()));
|
|
|
|
}
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
// getter
|
|
|
|
ReportFileOpCounters* counters() { return &counters_; }
|
|
|
|
|
|
|
|
private:
|
|
|
|
ReportFileOpCounters counters_;
|
|
|
|
};
|
|
|
|
|
|
|
|
} // namespace
|
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
// Helper for quickly generating random data.
|
2011-03-18 23:37:00 +01:00
|
|
|
class RandomGenerator {
|
|
|
|
private:
|
|
|
|
std::string data_;
|
2012-11-06 21:02:18 +01:00
|
|
|
unsigned int pos_;
|
2011-03-18 23:37:00 +01:00
|
|
|
|
|
|
|
public:
|
|
|
|
RandomGenerator() {
|
|
|
|
// We use a limited amount of data over and over again and ensure
|
|
|
|
// that it is larger than the compression window (32KB), and also
|
|
|
|
// large enough to serve all typical value sizes we want to write.
|
|
|
|
Random rnd(301);
|
|
|
|
std::string piece;
|
2013-03-15 02:32:01 +01:00
|
|
|
while (data_.size() < (unsigned)std::max(1048576, FLAGS_value_size)) {
|
2011-03-18 23:37:00 +01:00
|
|
|
// Add a short fragment that is as compressible as specified
|
|
|
|
// by FLAGS_compression_ratio.
|
|
|
|
test::CompressibleString(&rnd, FLAGS_compression_ratio, 100, &piece);
|
|
|
|
data_.append(piece);
|
|
|
|
}
|
|
|
|
pos_ = 0;
|
|
|
|
}
|
|
|
|
|
2013-03-15 02:32:01 +01:00
|
|
|
Slice Generate(unsigned int len) {
|
2014-06-24 23:29:28 +02:00
|
|
|
assert(len <= data_.size());
|
2011-03-18 23:37:00 +01:00
|
|
|
if (pos_ + len > data_.size()) {
|
|
|
|
pos_ = 0;
|
|
|
|
}
|
|
|
|
pos_ += len;
|
|
|
|
return Slice(data_.data() + pos_ - len, len);
|
|
|
|
}
|
2012-09-26 17:06:19 +02:00
|
|
|
};
|
2013-08-13 22:58:02 +02:00
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
static void AppendWithSpace(std::string* str, Slice msg) {
|
|
|
|
if (msg.empty()) return;
|
|
|
|
if (!str->empty()) {
|
|
|
|
str->push_back(' ');
|
|
|
|
}
|
|
|
|
str->append(msg.data(), msg.size());
|
|
|
|
}
|
|
|
|
|
2014-09-29 18:50:41 +02:00
|
|
|
struct DBWithColumnFamilies {
|
|
|
|
std::vector<ColumnFamilyHandle*> cfh;
|
|
|
|
DB* db;
|
2015-10-14 21:43:00 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
OptimisticTransactionDB* opt_txn_db;
|
2015-10-14 21:43:00 +02:00
|
|
|
#endif // ROCKSDB_LITE
|
2014-12-23 03:39:28 +01:00
|
|
|
std::atomic<size_t> num_created; // Need to be updated after all the
|
|
|
|
// new entries in cfh are set.
|
|
|
|
size_t num_hot; // Number of column families to be queried at each moment.
|
|
|
|
// After each CreateNewCf(), another num_hot number of new
|
|
|
|
// Column families will be created and used to be queried.
|
|
|
|
port::Mutex create_cf_mutex; // Only one thread can execute CreateNewCf()
|
|
|
|
|
2015-10-14 21:43:00 +02:00
|
|
|
DBWithColumnFamilies()
|
|
|
|
: db(nullptr)
|
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
, opt_txn_db(nullptr)
|
|
|
|
#endif // ROCKSDB_LITE
|
|
|
|
{
|
2014-09-29 18:50:41 +02:00
|
|
|
cfh.clear();
|
2015-03-12 01:57:16 +01:00
|
|
|
num_created = 0;
|
|
|
|
num_hot = 0;
|
2014-09-29 18:50:41 +02:00
|
|
|
}
|
2014-12-23 03:39:28 +01:00
|
|
|
|
|
|
|
DBWithColumnFamilies(const DBWithColumnFamilies& other)
|
|
|
|
: cfh(other.cfh),
|
|
|
|
db(other.db),
|
2015-10-14 21:43:00 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
opt_txn_db(other.opt_txn_db),
|
2015-10-14 21:43:00 +02:00
|
|
|
#endif // ROCKSDB_LITE
|
2014-12-23 03:39:28 +01:00
|
|
|
num_created(other.num_created.load()),
|
|
|
|
num_hot(other.num_hot) {}
|
|
|
|
|
2015-05-29 23:36:35 +02:00
|
|
|
void DeleteDBs() {
|
|
|
|
std::for_each(cfh.begin(), cfh.end(),
|
|
|
|
[](ColumnFamilyHandle* cfhi) { delete cfhi; });
|
|
|
|
cfh.clear();
|
2015-10-14 21:43:00 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
if (opt_txn_db) {
|
|
|
|
delete opt_txn_db;
|
|
|
|
opt_txn_db = nullptr;
|
2015-05-29 23:36:35 +02:00
|
|
|
} else {
|
|
|
|
delete db;
|
2015-10-14 21:43:00 +02:00
|
|
|
db = nullptr;
|
2015-05-29 23:36:35 +02:00
|
|
|
}
|
2015-10-14 21:43:00 +02:00
|
|
|
#else
|
|
|
|
delete db;
|
2015-05-29 23:36:35 +02:00
|
|
|
db = nullptr;
|
2015-10-14 21:43:00 +02:00
|
|
|
#endif // ROCKSDB_LITE
|
2015-05-29 23:36:35 +02:00
|
|
|
}
|
|
|
|
|
2014-12-23 03:39:28 +01:00
|
|
|
ColumnFamilyHandle* GetCfh(int64_t rand_num) {
|
|
|
|
assert(num_hot > 0);
|
|
|
|
return cfh[num_created.load(std::memory_order_acquire) - num_hot +
|
|
|
|
rand_num % num_hot];
|
|
|
|
}
|
|
|
|
|
|
|
|
// stage: assume CF from 0 to stage * num_hot has be created. Need to create
|
|
|
|
// stage * num_hot + 1 to stage * (num_hot + 1).
|
|
|
|
void CreateNewCf(ColumnFamilyOptions options, int64_t stage) {
|
|
|
|
MutexLock l(&create_cf_mutex);
|
|
|
|
if ((stage + 1) * num_hot <= num_created) {
|
|
|
|
// Already created.
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
auto new_num_created = num_created + num_hot;
|
|
|
|
assert(new_num_created <= cfh.size());
|
|
|
|
for (size_t i = num_created; i < new_num_created; i++) {
|
|
|
|
Status s =
|
|
|
|
db->CreateColumnFamily(options, ColumnFamilyName(i), &(cfh[i]));
|
|
|
|
if (!s.ok()) {
|
|
|
|
fprintf(stderr, "create column family error: %s\n",
|
|
|
|
s.ToString().c_str());
|
|
|
|
abort();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
num_created.store(new_num_created, std::memory_order_release);
|
|
|
|
}
|
2014-09-29 18:50:41 +02:00
|
|
|
};
|
|
|
|
|
db_bench periodically writes QPS to CSV file
Summary:
This is part of an effort to better understand and optimize RocksDB stalls under high load. I added a feature to db_bench to periodically write QPS to CSV files. That way we can nicely see how our QPS changes in time (especially when DB is stalled) and can do a better job of evaluating our stall system (i.e. we want the QPS to be as constant as possible, as opposed to having bunch of stalls)
Cool part of CSV files is that we can easily graph them -- there are a bunch of tools available.
Test Plan:
Ran ./db_bench --report_interval_seconds=10 --benchmarks=fillrandom --num=10000000
and observed this in report.csv:
secs_elapsed,interval_qps
10,2725860
20,1980480
30,1863456
40,1454359
50,1460389
Reviewers: sdong, MarkCallaghan, rven, yhchiang
Reviewed By: yhchiang
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D40047
2015-06-12 23:31:53 +02:00
|
|
|
// a class that reports stats to CSV file
|
|
|
|
class ReporterAgent {
|
|
|
|
public:
|
|
|
|
ReporterAgent(Env* env, const std::string& fname,
|
|
|
|
uint64_t report_interval_secs)
|
|
|
|
: env_(env),
|
|
|
|
total_ops_done_(0),
|
|
|
|
last_report_(0),
|
|
|
|
report_interval_secs_(report_interval_secs),
|
|
|
|
stop_(false) {
|
|
|
|
auto s = env_->NewWritableFile(fname, &report_file_, EnvOptions());
|
|
|
|
if (s.ok()) {
|
|
|
|
s = report_file_->Append(Header() + "\n");
|
|
|
|
}
|
|
|
|
if (s.ok()) {
|
|
|
|
s = report_file_->Flush();
|
|
|
|
}
|
|
|
|
if (!s.ok()) {
|
|
|
|
fprintf(stderr, "Can't open %s: %s\n", fname.c_str(),
|
|
|
|
s.ToString().c_str());
|
|
|
|
abort();
|
|
|
|
}
|
|
|
|
|
|
|
|
reporting_thread_ = std::thread([&]() { SleepAndReport(); });
|
|
|
|
}
|
|
|
|
|
|
|
|
~ReporterAgent() {
|
|
|
|
{
|
|
|
|
std::unique_lock<std::mutex> lk(mutex_);
|
|
|
|
stop_ = true;
|
|
|
|
stop_cv_.notify_all();
|
|
|
|
}
|
|
|
|
reporting_thread_.join();
|
|
|
|
}
|
|
|
|
|
|
|
|
// thread safe
|
|
|
|
void ReportFinishedOps(int64_t num_ops) {
|
|
|
|
total_ops_done_.fetch_add(num_ops);
|
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
std::string Header() const { return "secs_elapsed,interval_qps"; }
|
|
|
|
void SleepAndReport() {
|
|
|
|
uint64_t kMicrosInSecond = 1000 * 1000;
|
|
|
|
auto time_started = env_->NowMicros();
|
|
|
|
while (true) {
|
|
|
|
{
|
|
|
|
std::unique_lock<std::mutex> lk(mutex_);
|
|
|
|
if (stop_ ||
|
|
|
|
stop_cv_.wait_for(lk, std::chrono::seconds(report_interval_secs_),
|
|
|
|
[&]() { return stop_; })) {
|
|
|
|
// stopping
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
// else -> timeout, which means time for a report!
|
|
|
|
}
|
|
|
|
auto total_ops_done_snapshot = total_ops_done_.load();
|
|
|
|
// round the seconds elapsed
|
|
|
|
auto secs_elapsed =
|
|
|
|
(env_->NowMicros() - time_started + kMicrosInSecond / 2) /
|
|
|
|
kMicrosInSecond;
|
|
|
|
std::string report = ToString(secs_elapsed) + "," +
|
|
|
|
ToString(total_ops_done_snapshot - last_report_) +
|
|
|
|
"\n";
|
|
|
|
auto s = report_file_->Append(report);
|
|
|
|
if (s.ok()) {
|
|
|
|
s = report_file_->Flush();
|
|
|
|
}
|
|
|
|
if (!s.ok()) {
|
|
|
|
fprintf(stderr,
|
|
|
|
"Can't write to report file (%s), stopping the reporting\n",
|
|
|
|
s.ToString().c_str());
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
last_report_ = total_ops_done_snapshot;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
Env* env_;
|
|
|
|
std::unique_ptr<WritableFile> report_file_;
|
|
|
|
std::atomic<int64_t> total_ops_done_;
|
|
|
|
int64_t last_report_;
|
|
|
|
const uint64_t report_interval_secs_;
|
|
|
|
std::thread reporting_thread_;
|
|
|
|
std::mutex mutex_;
|
|
|
|
// will notify on stop
|
|
|
|
std::condition_variable stop_cv_;
|
|
|
|
bool stop_;
|
|
|
|
};
|
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
class Stats {
|
|
|
|
private:
|
2012-10-03 15:51:11 +02:00
|
|
|
int id_;
|
2011-08-22 23:08:51 +02:00
|
|
|
double start_;
|
|
|
|
double finish_;
|
|
|
|
double seconds_;
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t done_;
|
|
|
|
int64_t last_report_done_;
|
|
|
|
int64_t next_report_;
|
2011-08-22 23:08:51 +02:00
|
|
|
int64_t bytes_;
|
|
|
|
double last_op_finish_;
|
2012-10-03 15:51:11 +02:00
|
|
|
double last_report_finish_;
|
2013-02-15 20:53:17 +01:00
|
|
|
HistogramImpl hist_;
|
2011-08-22 23:08:51 +02:00
|
|
|
std::string message_;
|
2013-03-19 14:36:03 +01:00
|
|
|
bool exclude_from_merge_;
|
db_bench periodically writes QPS to CSV file
Summary:
This is part of an effort to better understand and optimize RocksDB stalls under high load. I added a feature to db_bench to periodically write QPS to CSV files. That way we can nicely see how our QPS changes in time (especially when DB is stalled) and can do a better job of evaluating our stall system (i.e. we want the QPS to be as constant as possible, as opposed to having bunch of stalls)
Cool part of CSV files is that we can easily graph them -- there are a bunch of tools available.
Test Plan:
Ran ./db_bench --report_interval_seconds=10 --benchmarks=fillrandom --num=10000000
and observed this in report.csv:
secs_elapsed,interval_qps
10,2725860
20,1980480
30,1863456
40,1454359
50,1460389
Reviewers: sdong, MarkCallaghan, rven, yhchiang
Reviewed By: yhchiang
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D40047
2015-06-12 23:31:53 +02:00
|
|
|
ReporterAgent* reporter_agent_; // does not own
|
2011-08-22 23:08:51 +02:00
|
|
|
|
|
|
|
public:
|
2012-10-03 15:51:11 +02:00
|
|
|
Stats() { Start(-1); }
|
2011-08-22 23:08:51 +02:00
|
|
|
|
db_bench periodically writes QPS to CSV file
Summary:
This is part of an effort to better understand and optimize RocksDB stalls under high load. I added a feature to db_bench to periodically write QPS to CSV files. That way we can nicely see how our QPS changes in time (especially when DB is stalled) and can do a better job of evaluating our stall system (i.e. we want the QPS to be as constant as possible, as opposed to having bunch of stalls)
Cool part of CSV files is that we can easily graph them -- there are a bunch of tools available.
Test Plan:
Ran ./db_bench --report_interval_seconds=10 --benchmarks=fillrandom --num=10000000
and observed this in report.csv:
secs_elapsed,interval_qps
10,2725860
20,1980480
30,1863456
40,1454359
50,1460389
Reviewers: sdong, MarkCallaghan, rven, yhchiang
Reviewed By: yhchiang
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D40047
2015-06-12 23:31:53 +02:00
|
|
|
void SetReporterAgent(ReporterAgent* reporter_agent) {
|
|
|
|
reporter_agent_ = reporter_agent;
|
|
|
|
}
|
|
|
|
|
2012-10-03 15:51:11 +02:00
|
|
|
void Start(int id) {
|
|
|
|
id_ = id;
|
|
|
|
next_report_ = FLAGS_stats_interval ? FLAGS_stats_interval : 100;
|
2011-08-22 23:08:51 +02:00
|
|
|
last_op_finish_ = start_;
|
|
|
|
hist_.Clear();
|
|
|
|
done_ = 0;
|
2012-10-03 15:51:11 +02:00
|
|
|
last_report_done_ = 0;
|
2011-08-22 23:08:51 +02:00
|
|
|
bytes_ = 0;
|
|
|
|
seconds_ = 0;
|
2012-06-08 10:11:14 +02:00
|
|
|
start_ = FLAGS_env->NowMicros();
|
2011-08-22 23:08:51 +02:00
|
|
|
finish_ = start_;
|
2012-10-03 15:51:11 +02:00
|
|
|
last_report_finish_ = start_;
|
2011-08-22 23:08:51 +02:00
|
|
|
message_.clear();
|
2013-03-19 14:36:03 +01:00
|
|
|
// When set, stats from this thread won't be merged with others.
|
|
|
|
exclude_from_merge_ = false;
|
2011-08-22 23:08:51 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
void Merge(const Stats& other) {
|
2013-03-19 14:36:03 +01:00
|
|
|
if (other.exclude_from_merge_)
|
|
|
|
return;
|
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
hist_.Merge(other.hist_);
|
|
|
|
done_ += other.done_;
|
|
|
|
bytes_ += other.bytes_;
|
|
|
|
seconds_ += other.seconds_;
|
|
|
|
if (other.start_ < start_) start_ = other.start_;
|
|
|
|
if (other.finish_ > finish_) finish_ = other.finish_;
|
|
|
|
|
|
|
|
// Just keep the messages from one thread
|
|
|
|
if (message_.empty()) message_ = other.message_;
|
|
|
|
}
|
|
|
|
|
|
|
|
void Stop() {
|
2012-06-08 10:11:14 +02:00
|
|
|
finish_ = FLAGS_env->NowMicros();
|
2011-08-22 23:08:51 +02:00
|
|
|
seconds_ = (finish_ - start_) * 1e-6;
|
|
|
|
}
|
|
|
|
|
|
|
|
void AddMessage(Slice msg) {
|
|
|
|
AppendWithSpace(&message_, msg);
|
|
|
|
}
|
|
|
|
|
2012-10-03 15:51:11 +02:00
|
|
|
void SetId(int id) { id_ = id; }
|
2013-03-19 14:36:03 +01:00
|
|
|
void SetExcludeFromMerge() { exclude_from_merge_ = true; }
|
2012-10-03 15:51:11 +02:00
|
|
|
|
2015-03-06 20:22:06 +01:00
|
|
|
void PrintThreadStatus() {
|
|
|
|
std::vector<ThreadStatus> thread_list;
|
|
|
|
FLAGS_env->GetThreadList(&thread_list);
|
|
|
|
|
2015-05-07 07:50:35 +02:00
|
|
|
fprintf(stderr, "\n%18s %10s %12s %20s %13s %45s %12s %s\n",
|
2015-03-10 22:51:28 +01:00
|
|
|
"ThreadID", "ThreadType", "cfName", "Operation",
|
2015-05-07 07:50:35 +02:00
|
|
|
"ElapsedTime", "Stage", "State", "OperationProperties");
|
2015-03-06 20:22:06 +01:00
|
|
|
|
2015-03-10 22:51:28 +01:00
|
|
|
int64_t current_time = 0;
|
|
|
|
Env::Default()->GetCurrentTime(¤t_time);
|
2015-03-06 20:22:06 +01:00
|
|
|
for (auto ts : thread_list) {
|
2015-05-07 07:50:35 +02:00
|
|
|
fprintf(stderr, "%18" PRIu64 " %10s %12s %20s %13s %45s %12s",
|
2015-03-06 20:22:06 +01:00
|
|
|
ts.thread_id,
|
|
|
|
ThreadStatus::GetThreadTypeName(ts.thread_type).c_str(),
|
|
|
|
ts.cf_name.c_str(),
|
|
|
|
ThreadStatus::GetOperationName(ts.operation_type).c_str(),
|
2015-03-24 00:35:04 +01:00
|
|
|
ThreadStatus::MicrosToString(ts.op_elapsed_micros).c_str(),
|
2015-03-13 18:45:40 +01:00
|
|
|
ThreadStatus::GetOperationStageName(ts.operation_stage).c_str(),
|
2015-03-06 20:22:06 +01:00
|
|
|
ThreadStatus::GetStateName(ts.state_type).c_str());
|
2015-05-07 07:50:35 +02:00
|
|
|
|
|
|
|
auto op_properties = ThreadStatus::InterpretOperationProperties(
|
|
|
|
ts.operation_type, ts.op_properties);
|
|
|
|
for (const auto& op_prop : op_properties) {
|
|
|
|
fprintf(stderr, " %s %" PRIu64" |",
|
|
|
|
op_prop.first.c_str(), op_prop.second);
|
|
|
|
}
|
|
|
|
fprintf(stderr, "\n");
|
2015-03-06 20:22:06 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-09-29 18:50:41 +02:00
|
|
|
void FinishedOps(DBWithColumnFamilies* db_with_cfh, DB* db, int64_t num_ops) {
|
db_bench periodically writes QPS to CSV file
Summary:
This is part of an effort to better understand and optimize RocksDB stalls under high load. I added a feature to db_bench to periodically write QPS to CSV files. That way we can nicely see how our QPS changes in time (especially when DB is stalled) and can do a better job of evaluating our stall system (i.e. we want the QPS to be as constant as possible, as opposed to having bunch of stalls)
Cool part of CSV files is that we can easily graph them -- there are a bunch of tools available.
Test Plan:
Ran ./db_bench --report_interval_seconds=10 --benchmarks=fillrandom --num=10000000
and observed this in report.csv:
secs_elapsed,interval_qps
10,2725860
20,1980480
30,1863456
40,1454359
50,1460389
Reviewers: sdong, MarkCallaghan, rven, yhchiang
Reviewed By: yhchiang
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D40047
2015-06-12 23:31:53 +02:00
|
|
|
if (reporter_agent_) {
|
|
|
|
reporter_agent_->ReportFinishedOps(num_ops);
|
|
|
|
}
|
2011-08-22 23:08:51 +02:00
|
|
|
if (FLAGS_histogram) {
|
2012-06-08 10:11:14 +02:00
|
|
|
double now = FLAGS_env->NowMicros();
|
2011-08-22 23:08:51 +02:00
|
|
|
double micros = now - last_op_finish_;
|
|
|
|
hist_.Add(micros);
|
2012-10-03 15:51:11 +02:00
|
|
|
if (micros > 20000 && !FLAGS_stats_interval) {
|
2011-08-22 23:08:51 +02:00
|
|
|
fprintf(stderr, "long op: %.1f micros%30s\r", micros, "");
|
|
|
|
fflush(stderr);
|
|
|
|
}
|
|
|
|
last_op_finish_ = now;
|
|
|
|
}
|
|
|
|
|
2014-06-23 22:23:02 +02:00
|
|
|
done_ += num_ops;
|
2011-08-22 23:08:51 +02:00
|
|
|
if (done_ >= next_report_) {
|
2012-10-03 15:51:11 +02:00
|
|
|
if (!FLAGS_stats_interval) {
|
|
|
|
if (next_report_ < 1000) next_report_ += 100;
|
|
|
|
else if (next_report_ < 5000) next_report_ += 500;
|
|
|
|
else if (next_report_ < 10000) next_report_ += 1000;
|
|
|
|
else if (next_report_ < 50000) next_report_ += 5000;
|
|
|
|
else if (next_report_ < 100000) next_report_ += 10000;
|
|
|
|
else if (next_report_ < 500000) next_report_ += 50000;
|
|
|
|
else next_report_ += 100000;
|
2014-03-05 02:08:05 +01:00
|
|
|
fprintf(stderr, "... finished %" PRIu64 " ops%30s\r", done_, "");
|
2012-10-03 15:51:11 +02:00
|
|
|
} else {
|
|
|
|
double now = FLAGS_env->NowMicros();
|
2015-03-30 21:58:32 +02:00
|
|
|
int64_t usecs_since_last = now - last_report_finish_;
|
|
|
|
|
|
|
|
// Determine whether to print status where interval is either
|
|
|
|
// each N operations or each N seconds.
|
|
|
|
|
|
|
|
if (FLAGS_stats_interval_seconds &&
|
|
|
|
usecs_since_last < (FLAGS_stats_interval_seconds * 1000000)) {
|
|
|
|
// Don't check again for this many operations
|
|
|
|
next_report_ += FLAGS_stats_interval;
|
|
|
|
|
|
|
|
} else {
|
2014-09-29 18:50:41 +02:00
|
|
|
|
2015-03-30 21:58:32 +02:00
|
|
|
fprintf(stderr,
|
|
|
|
"%s ... thread %d: (%" PRIu64 ",%" PRIu64 ") ops and "
|
|
|
|
"(%.1f,%.1f) ops/second in (%.6f,%.6f) seconds\n",
|
|
|
|
FLAGS_env->TimeToString((uint64_t) now/1000000).c_str(),
|
|
|
|
id_,
|
|
|
|
done_ - last_report_done_, done_,
|
|
|
|
(done_ - last_report_done_) /
|
|
|
|
(usecs_since_last / 1000000.0),
|
|
|
|
done_ / ((now - start_) / 1000000.0),
|
|
|
|
(now - last_report_finish_) / 1000000.0,
|
|
|
|
(now - start_) / 1000000.0);
|
|
|
|
|
|
|
|
if (FLAGS_stats_per_interval) {
|
|
|
|
std::string stats;
|
|
|
|
|
|
|
|
if (db_with_cfh && db_with_cfh->num_created.load()) {
|
|
|
|
for (size_t i = 0; i < db_with_cfh->num_created.load(); ++i) {
|
|
|
|
if (db->GetProperty(db_with_cfh->cfh[i], "rocksdb.cfstats",
|
|
|
|
&stats))
|
|
|
|
fprintf(stderr, "%s\n", stats.c_str());
|
Add argument --show_table_properties to db_bench
Summary:
Add argument --show_table_properties to db_bench
-show_table_properties (If true, then per-level table properties will be
printed on every stats-interval when stats_interval is set and
stats_per_interval is on.) type: bool default: false
Test Plan:
./db_bench --show_table_properties=1 --stats_interval=100000 --stats_per_interval=1
./db_bench --show_table_properties=1 --stats_interval=100000 --stats_per_interval=1 --num_column_families=2
Sample Output:
Compaction Stats [column_family_name_000001]
Level Files Size(MB) Score Read(GB) Rn(GB) Rnp1(GB) Write(GB) Wnew(GB) Moved(GB) W-Amp Rd(MB/s) Wr(MB/s) Comp(sec) Comp(cnt) Avg(sec) Stall(cnt) KeyIn KeyDrop
---------------------------------------------------------------------------------------------------------------------------------------------------------------------
L0 3/0 5 0.8 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 86.3 0 17 0.021 0 0 0
L1 5/0 9 0.9 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0 0 0.000 0 0 0
L2 9/0 16 0.2 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0 0 0.000 0 0 0
Sum 17/0 31 0.0 0.0 0.0 0.0 0.0 0.0 0.0 1.0 0.0 86.3 0 17 0.021 0 0 0
Int 0/0 0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 1.0 0.0 83.9 0 2 0.022 0 0 0
Flush(GB): cumulative 0.030, interval 0.004
Stalls(count): 0 level0_slowdown, 0 level0_numfiles, 0 memtable_compaction, 0 leveln_slowdown_soft, 0 leveln_slowdown_hard
Level[0]: # data blocks=2571; # entries=84813; raw key size=2035512; raw average key size=24.000000; raw value size=8481300; raw average value size=100.000000; data block size=5690119; index block size=82415; filter block size=0; (estimated) table size=5772534; filter policy name=N/A;
Level[1]: # data blocks=4285; # entries=141355; raw key size=3392520; raw average key size=24.000000; raw value size=14135500; raw average value size=100.000000; data block size=9487353; index block size=137377; filter block size=0; (estimated) table size=9624730; filter policy name=N/A;
Level[2]: # data blocks=7713; # entries=254439; raw key size=6106536; raw average key size=24.000000; raw value size=25443900; raw average value size=100.000000; data block size=17077893; index block size=247269; filter block size=0; (estimated) table size=17325162; filter policy name=N/A;
Level[3]: # data blocks=0; # entries=0; raw key size=0; raw average key size=0.000000; raw value size=0; raw average value size=0.000000; data block size=0; index block size=0; filter block size=0; (estimated) table size=0; filter policy name=N/A;
Level[4]: # data blocks=0; # entries=0; raw key size=0; raw average key size=0.000000; raw value size=0; raw average value size=0.000000; data block size=0; index block size=0; filter block size=0; (estimated) table size=0; filter policy name=N/A;
Level[5]: # data blocks=0; # entries=0; raw key size=0; raw average key size=0.000000; raw value size=0; raw average value size=0.000000; data block size=0; index block size=0; filter block size=0; (estimated) table size=0; filter policy name=N/A;
Level[6]: # data blocks=0; # entries=0; raw key size=0; raw average key size=0.000000; raw value size=0; raw average value size=0.000000; data block size=0; index block size=0; filter block size=0; (estimated) table size=0; filter policy name=N/A;
Reviewers: anthony, IslamAbdelRahman, MarkCallaghan, sdong, igor
Reviewed By: igor
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D45651
2015-08-27 03:27:23 +02:00
|
|
|
if (FLAGS_show_table_properties) {
|
|
|
|
for (int level = 0; level < FLAGS_num_levels; ++level) {
|
|
|
|
if (db->GetProperty(
|
|
|
|
db_with_cfh->cfh[i],
|
|
|
|
"rocksdb.aggregated-table-properties-at-level" +
|
|
|
|
ToString(level),
|
|
|
|
&stats)) {
|
|
|
|
if (stats.find("# entries=0") == std::string::npos) {
|
|
|
|
fprintf(stderr, "Level[%d]: %s\n", level,
|
|
|
|
stats.c_str());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
} else if (db) {
|
|
|
|
if (db->GetProperty("rocksdb.stats", &stats)) {
|
|
|
|
fprintf(stderr, "%s\n", stats.c_str());
|
|
|
|
}
|
|
|
|
if (FLAGS_show_table_properties) {
|
|
|
|
for (int level = 0; level < FLAGS_num_levels; ++level) {
|
|
|
|
if (db->GetProperty(
|
|
|
|
"rocksdb.aggregated-table-properties-at-level" +
|
|
|
|
ToString(level),
|
|
|
|
&stats)) {
|
|
|
|
if (stats.find("# entries=0") == std::string::npos) {
|
|
|
|
fprintf(stderr, "Level[%d]: %s\n", level, stats.c_str());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2015-03-30 21:58:32 +02:00
|
|
|
}
|
|
|
|
}
|
2014-09-29 18:50:41 +02:00
|
|
|
}
|
2012-10-23 19:34:09 +02:00
|
|
|
|
2015-03-30 21:58:32 +02:00
|
|
|
next_report_ += FLAGS_stats_interval;
|
|
|
|
last_report_finish_ = now;
|
|
|
|
last_report_done_ = done_;
|
|
|
|
}
|
2012-10-03 15:51:11 +02:00
|
|
|
}
|
2015-03-06 20:22:06 +01:00
|
|
|
if (id_ == 0 && FLAGS_thread_status_per_interval) {
|
|
|
|
PrintThreadStatus();
|
|
|
|
}
|
|
|
|
fflush(stderr);
|
2011-08-22 23:08:51 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void AddBytes(int64_t n) {
|
|
|
|
bytes_ += n;
|
|
|
|
}
|
|
|
|
|
|
|
|
void Report(const Slice& name) {
|
|
|
|
// Pretend at least one op was done in case we are running a benchmark
|
2014-06-23 22:23:02 +02:00
|
|
|
// that does not call FinishedOps().
|
2011-08-22 23:08:51 +02:00
|
|
|
if (done_ < 1) done_ = 1;
|
|
|
|
|
|
|
|
std::string extra;
|
|
|
|
if (bytes_ > 0) {
|
|
|
|
// Rate is computed on actual elapsed time, not the sum of per-thread
|
|
|
|
// elapsed times.
|
|
|
|
double elapsed = (finish_ - start_) * 1e-6;
|
|
|
|
char rate[100];
|
|
|
|
snprintf(rate, sizeof(rate), "%6.1f MB/s",
|
|
|
|
(bytes_ / 1048576.0) / elapsed);
|
|
|
|
extra = rate;
|
|
|
|
}
|
|
|
|
AppendWithSpace(&extra, message_);
|
2012-05-11 20:32:37 +02:00
|
|
|
double elapsed = (finish_ - start_) * 1e-6;
|
|
|
|
double throughput = (double)done_/elapsed;
|
2011-08-22 23:08:51 +02:00
|
|
|
|
2012-05-10 08:32:20 +02:00
|
|
|
fprintf(stdout, "%-12s : %11.3f micros/op %ld ops/sec;%s%s\n",
|
2011-08-22 23:08:51 +02:00
|
|
|
name.ToString().c_str(),
|
2013-06-12 21:42:21 +02:00
|
|
|
elapsed * 1e6 / done_,
|
2012-05-10 08:32:20 +02:00
|
|
|
(long)throughput,
|
2011-08-22 23:08:51 +02:00
|
|
|
(extra.empty() ? "" : " "),
|
|
|
|
extra.c_str());
|
|
|
|
if (FLAGS_histogram) {
|
|
|
|
fprintf(stdout, "Microseconds per op:\n%s\n", hist_.ToString().c_str());
|
|
|
|
}
|
2014-10-29 22:24:34 +01:00
|
|
|
if (FLAGS_report_file_operations) {
|
|
|
|
ReportFileOpEnv* env = static_cast<ReportFileOpEnv*>(FLAGS_env);
|
|
|
|
ReportFileOpCounters* counters = env->counters();
|
|
|
|
fprintf(stdout, "Num files opened: %d\n",
|
|
|
|
counters->open_counter_.load(std::memory_order_relaxed));
|
|
|
|
fprintf(stdout, "Num Read(): %d\n",
|
|
|
|
counters->read_counter_.load(std::memory_order_relaxed));
|
|
|
|
fprintf(stdout, "Num Append(): %d\n",
|
|
|
|
counters->append_counter_.load(std::memory_order_relaxed));
|
|
|
|
fprintf(stdout, "Num bytes read: %" PRIu64 "\n",
|
|
|
|
counters->bytes_read_.load(std::memory_order_relaxed));
|
|
|
|
fprintf(stdout, "Num bytes written: %" PRIu64 "\n",
|
|
|
|
counters->bytes_written_.load(std::memory_order_relaxed));
|
|
|
|
env->reset();
|
|
|
|
}
|
2011-08-22 23:08:51 +02:00
|
|
|
fflush(stdout);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
// State shared by all concurrent executions of the same benchmark.
|
|
|
|
struct SharedState {
|
|
|
|
port::Mutex mu;
|
|
|
|
port::CondVar cv;
|
|
|
|
int total;
|
2013-11-18 20:32:54 +01:00
|
|
|
int perf_level;
|
2015-06-18 01:44:52 +02:00
|
|
|
std::shared_ptr<RateLimiter> write_rate_limiter;
|
2011-08-22 23:08:51 +02:00
|
|
|
|
|
|
|
// Each thread goes through the following states:
|
|
|
|
// (1) initializing
|
|
|
|
// (2) waiting for others to be initialized
|
|
|
|
// (3) running
|
|
|
|
// (4) done
|
|
|
|
|
2012-05-22 21:18:23 +02:00
|
|
|
long num_initialized;
|
|
|
|
long num_done;
|
2011-08-22 23:08:51 +02:00
|
|
|
bool start;
|
|
|
|
|
2013-11-18 20:32:54 +01:00
|
|
|
SharedState() : cv(&mu), perf_level(FLAGS_perf_level) { }
|
2011-08-22 23:08:51 +02:00
|
|
|
};
|
|
|
|
|
|
|
|
// Per-thread state for concurrent executions of the same benchmark.
|
|
|
|
struct ThreadState {
|
|
|
|
int tid; // 0..n-1 when running in n threads
|
2013-08-23 07:37:13 +02:00
|
|
|
Random64 rand; // Has different seeds for different threads
|
2011-08-22 23:08:51 +02:00
|
|
|
Stats stats;
|
2011-09-01 21:08:02 +02:00
|
|
|
SharedState* shared;
|
2011-08-22 23:08:51 +02:00
|
|
|
|
2012-11-29 01:42:36 +01:00
|
|
|
/* implicit */ ThreadState(int index)
|
2011-08-22 23:08:51 +02:00
|
|
|
: tid(index),
|
2013-01-03 21:11:50 +01:00
|
|
|
rand((FLAGS_seed ? FLAGS_seed : 1000) + index) {
|
2011-08-22 23:08:51 +02:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2013-03-01 20:25:13 +01:00
|
|
|
class Duration {
|
|
|
|
public:
|
2014-12-23 03:39:28 +01:00
|
|
|
Duration(int max_seconds, int64_t max_ops, int64_t ops_per_stage = 0) {
|
2013-03-01 20:25:13 +01:00
|
|
|
max_seconds_ = max_seconds;
|
|
|
|
max_ops_= max_ops;
|
2014-12-23 03:39:28 +01:00
|
|
|
ops_per_stage_ = (ops_per_stage > 0) ? ops_per_stage : max_ops;
|
2013-03-01 20:25:13 +01:00
|
|
|
ops_ = 0;
|
|
|
|
start_at_ = FLAGS_env->NowMicros();
|
|
|
|
}
|
|
|
|
|
2014-12-23 03:39:28 +01:00
|
|
|
int64_t GetStage() { return std::min(ops_, max_ops_ - 1) / ops_per_stage_; }
|
|
|
|
|
2014-04-08 20:21:09 +02:00
|
|
|
bool Done(int64_t increment) {
|
2013-06-12 21:42:21 +02:00
|
|
|
if (increment <= 0) increment = 1; // avoid Done(0) and infinite loops
|
2013-03-01 20:25:13 +01:00
|
|
|
ops_ += increment;
|
|
|
|
|
|
|
|
if (max_seconds_) {
|
2013-06-12 21:42:21 +02:00
|
|
|
// Recheck every appx 1000 ops (exact iff increment is factor of 1000)
|
|
|
|
if ((ops_/1000) != ((ops_-increment)/1000)) {
|
2013-03-01 20:25:13 +01:00
|
|
|
double now = FLAGS_env->NowMicros();
|
|
|
|
return ((now - start_at_) / 1000000.0) >= max_seconds_;
|
|
|
|
} else {
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
return ops_ > max_ops_;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
int max_seconds_;
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t max_ops_;
|
2014-12-23 03:39:28 +01:00
|
|
|
int64_t ops_per_stage_;
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t ops_;
|
2013-03-01 20:25:13 +01:00
|
|
|
double start_at_;
|
|
|
|
};
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
class Benchmark {
|
|
|
|
private:
|
2014-08-25 23:22:05 +02:00
|
|
|
std::shared_ptr<Cache> cache_;
|
|
|
|
std::shared_ptr<Cache> compressed_cache_;
|
|
|
|
std::shared_ptr<const FilterPolicy> filter_policy_;
|
2013-08-23 23:49:57 +02:00
|
|
|
const SliceTransform* prefix_extractor_;
|
2014-08-19 03:15:01 +02:00
|
|
|
DBWithColumnFamilies db_;
|
|
|
|
std::vector<DBWithColumnFamilies> multi_dbs_;
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t num_;
|
2011-08-22 23:08:51 +02:00
|
|
|
int value_size_;
|
2013-03-01 23:10:09 +01:00
|
|
|
int key_size_;
|
2014-03-05 02:08:05 +01:00
|
|
|
int prefix_size_;
|
|
|
|
int64_t keys_per_prefix_;
|
2014-04-08 20:21:09 +02:00
|
|
|
int64_t entries_per_batch_;
|
2011-08-22 23:08:51 +02:00
|
|
|
WriteOptions write_options_;
|
2014-12-23 03:39:28 +01:00
|
|
|
Options open_options_; // keep options around to properly destroy db later
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t reads_;
|
2015-03-07 01:38:50 +01:00
|
|
|
double read_random_exp_range_;
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t writes_;
|
|
|
|
int64_t readwrites_;
|
|
|
|
int64_t merge_keys_;
|
2014-10-29 22:24:34 +01:00
|
|
|
bool report_file_operations_;
|
2015-04-03 03:14:49 +02:00
|
|
|
int cachedev_fd_;
|
2014-06-23 19:46:16 +02:00
|
|
|
|
|
|
|
bool SanityCheck() {
|
|
|
|
if (FLAGS_compression_ratio > 1) {
|
|
|
|
fprintf(stderr, "compression_ratio should be between 0 and 1\n");
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2015-08-11 20:46:15 +02:00
|
|
|
inline bool CompressSlice(const Slice& input, std::string* compressed) {
|
|
|
|
bool ok = true;
|
|
|
|
switch (FLAGS_compression_type_e) {
|
|
|
|
case rocksdb::kSnappyCompression:
|
|
|
|
ok = Snappy_Compress(Options().compression_opts, input.data(),
|
|
|
|
input.size(), compressed);
|
|
|
|
break;
|
|
|
|
case rocksdb::kZlibCompression:
|
|
|
|
ok = Zlib_Compress(Options().compression_opts, 2, input.data(),
|
|
|
|
input.size(), compressed);
|
|
|
|
break;
|
|
|
|
case rocksdb::kBZip2Compression:
|
|
|
|
ok = BZip2_Compress(Options().compression_opts, 2, input.data(),
|
|
|
|
input.size(), compressed);
|
|
|
|
break;
|
|
|
|
case rocksdb::kLZ4Compression:
|
|
|
|
ok = LZ4_Compress(Options().compression_opts, 2, input.data(),
|
|
|
|
input.size(), compressed);
|
|
|
|
break;
|
|
|
|
case rocksdb::kLZ4HCCompression:
|
|
|
|
ok = LZ4HC_Compress(Options().compression_opts, 2, input.data(),
|
|
|
|
input.size(), compressed);
|
|
|
|
break;
|
2015-08-28 00:40:42 +02:00
|
|
|
case rocksdb::kZSTDNotFinalCompression:
|
|
|
|
ok = ZSTD_Compress(Options().compression_opts, input.data(),
|
|
|
|
input.size(), compressed);
|
|
|
|
break;
|
2015-08-11 20:46:15 +02:00
|
|
|
default:
|
|
|
|
ok = false;
|
|
|
|
}
|
|
|
|
return ok;
|
|
|
|
}
|
|
|
|
|
2011-03-22 19:32:49 +01:00
|
|
|
void PrintHeader() {
|
|
|
|
PrintEnvironment();
|
2013-03-01 23:10:09 +01:00
|
|
|
fprintf(stdout, "Keys: %d bytes each\n", FLAGS_key_size);
|
2011-03-22 19:32:49 +01:00
|
|
|
fprintf(stdout, "Values: %d bytes each (%d bytes after compression)\n",
|
|
|
|
FLAGS_value_size,
|
|
|
|
static_cast<int>(FLAGS_value_size * FLAGS_compression_ratio + 0.5));
|
2014-03-05 02:08:05 +01:00
|
|
|
fprintf(stdout, "Entries: %" PRIu64 "\n", num_);
|
|
|
|
fprintf(stdout, "Prefix: %d bytes\n", FLAGS_prefix_size);
|
|
|
|
fprintf(stdout, "Keys per prefix: %" PRIu64 "\n", keys_per_prefix_);
|
2011-03-22 19:32:49 +01:00
|
|
|
fprintf(stdout, "RawSize: %.1f MB (estimated)\n",
|
2013-03-01 23:10:09 +01:00
|
|
|
((static_cast<int64_t>(FLAGS_key_size + FLAGS_value_size) * num_)
|
2011-04-12 21:38:58 +02:00
|
|
|
/ 1048576.0));
|
2011-03-22 19:32:49 +01:00
|
|
|
fprintf(stdout, "FileSize: %.1f MB (estimated)\n",
|
2013-10-24 16:43:14 +02:00
|
|
|
(((FLAGS_key_size + FLAGS_value_size * FLAGS_compression_ratio)
|
|
|
|
* num_)
|
2011-03-22 19:32:49 +01:00
|
|
|
/ 1048576.0));
|
2015-06-18 01:44:52 +02:00
|
|
|
fprintf(stdout, "Writes per second: %d\n", FLAGS_writes_per_second);
|
2014-07-07 19:53:31 +02:00
|
|
|
if (FLAGS_enable_numa) {
|
|
|
|
fprintf(stderr, "Running in NUMA enabled mode.\n");
|
|
|
|
#ifndef NUMA
|
|
|
|
fprintf(stderr, "NUMA is not defined in the system.\n");
|
|
|
|
exit(1);
|
|
|
|
#else
|
|
|
|
if (numa_available() == -1) {
|
|
|
|
fprintf(stderr, "NUMA is not supported by the system.\n");
|
|
|
|
exit(1);
|
|
|
|
}
|
|
|
|
#endif
|
|
|
|
}
|
2015-08-11 20:46:15 +02:00
|
|
|
|
|
|
|
const char* compression =
|
|
|
|
CompressionTypeToString(FLAGS_compression_type_e).c_str();
|
|
|
|
fprintf(stdout, "Compression: %s\n", compression);
|
2012-09-14 20:21:54 +02:00
|
|
|
|
2013-08-23 08:10:02 +02:00
|
|
|
switch (FLAGS_rep_factory) {
|
|
|
|
case kPrefixHash:
|
|
|
|
fprintf(stdout, "Memtablerep: prefix_hash\n");
|
|
|
|
break;
|
|
|
|
case kSkipList:
|
|
|
|
fprintf(stdout, "Memtablerep: skip_list\n");
|
|
|
|
break;
|
|
|
|
case kVectorRep:
|
|
|
|
fprintf(stdout, "Memtablerep: vector\n");
|
|
|
|
break;
|
2014-03-05 19:27:17 +01:00
|
|
|
case kHashLinkedList:
|
|
|
|
fprintf(stdout, "Memtablerep: hash_linkedlist\n");
|
|
|
|
break;
|
Add a new mem-table representation based on cuckoo hash.
Summary:
= Major Changes =
* Add a new mem-table representation, HashCuckooRep, which is based cuckoo hash.
Cuckoo hash uses multiple hash functions. This allows each key to have multiple
possible locations in the mem-table.
- Put: When insert a key, it will try to find whether one of its possible
locations is vacant and store the key. If none of its possible
locations are available, then it will kick out a victim key and
store at that location. The kicked-out victim key will then be
stored at a vacant space of its possible locations or kick-out
another victim. In this diff, the kick-out path (known as
cuckoo-path) is found using BFS, which guarantees to be the shortest.
- Get: Simply tries all possible locations of a key --- this guarantees
worst-case constant time complexity.
- Time complexity: O(1) for Get, and average O(1) for Put if the
fullness of the mem-table is below 80%.
- Default using two hash functions, the number of hash functions used
by the cuckoo-hash may dynamically increase if it fails to find a
short-enough kick-out path.
- Currently, HashCuckooRep does not support iteration and snapshots,
as our current main purpose of this is to optimize point access.
= Minor Changes =
* Add IsSnapshotSupported() to DB to indicate whether the current DB
supports snapshots. If it returns false, then DB::GetSnapshot() will
always return nullptr.
Test Plan:
Run existing tests. Will develop a test specifically for cuckoo hash in
the next diff.
Reviewers: sdong, haobo
Reviewed By: sdong
CC: leveldb, dhruba, igor
Differential Revision: https://reviews.facebook.net/D16155
2014-04-30 02:13:46 +02:00
|
|
|
case kCuckoo:
|
|
|
|
fprintf(stdout, "Memtablerep: cuckoo\n");
|
|
|
|
break;
|
2013-08-23 08:10:02 +02:00
|
|
|
}
|
2013-11-18 20:32:54 +01:00
|
|
|
fprintf(stdout, "Perf Level: %d\n", FLAGS_perf_level);
|
2013-08-23 08:10:02 +02:00
|
|
|
|
2015-08-11 20:46:15 +02:00
|
|
|
PrintWarnings(compression);
|
2011-03-22 19:32:49 +01:00
|
|
|
fprintf(stdout, "------------------------------------------------\n");
|
|
|
|
}
|
|
|
|
|
2015-08-11 20:46:15 +02:00
|
|
|
void PrintWarnings(const char* compression) {
|
2011-03-22 19:32:49 +01:00
|
|
|
#if defined(__GNUC__) && !defined(__OPTIMIZE__)
|
|
|
|
fprintf(stdout,
|
|
|
|
"WARNING: Optimization is disabled: benchmarks unnecessarily slow\n"
|
|
|
|
);
|
|
|
|
#endif
|
|
|
|
#ifndef NDEBUG
|
|
|
|
fprintf(stdout,
|
|
|
|
"WARNING: Assertions are enabled; benchmarks unnecessarily slow\n");
|
|
|
|
#endif
|
2013-10-24 16:43:14 +02:00
|
|
|
if (FLAGS_compression_type_e != rocksdb::kNoCompression) {
|
2012-09-14 20:21:54 +02:00
|
|
|
// The test string should not be too small.
|
|
|
|
const int len = FLAGS_block_size;
|
2015-08-11 20:46:15 +02:00
|
|
|
std::string input_str(len, 'y');
|
2012-09-14 20:21:54 +02:00
|
|
|
std::string compressed;
|
2015-08-11 20:46:15 +02:00
|
|
|
bool result = CompressSlice(Slice(input_str), &compressed);
|
2012-09-14 20:21:54 +02:00
|
|
|
|
|
|
|
if (!result) {
|
2015-08-11 20:46:15 +02:00
|
|
|
fprintf(stdout, "WARNING: %s compression is not enabled\n",
|
|
|
|
compression);
|
|
|
|
} else if (compressed.size() >= input_str.size()) {
|
|
|
|
fprintf(stdout, "WARNING: %s compression is not effective\n",
|
|
|
|
compression);
|
2012-09-14 20:21:54 +02:00
|
|
|
}
|
2011-04-12 21:38:58 +02:00
|
|
|
}
|
2011-03-22 19:32:49 +01:00
|
|
|
}
|
|
|
|
|
2013-11-17 08:44:39 +01:00
|
|
|
// Current the following isn't equivalent to OS_LINUX.
|
|
|
|
#if defined(__linux)
|
|
|
|
static Slice TrimSpace(Slice s) {
|
|
|
|
unsigned int start = 0;
|
|
|
|
while (start < s.size() && isspace(s[start])) {
|
|
|
|
start++;
|
|
|
|
}
|
2014-11-21 20:20:42 +01:00
|
|
|
unsigned int limit = static_cast<unsigned int>(s.size());
|
2013-11-17 08:44:39 +01:00
|
|
|
while (limit > start && isspace(s[limit-1])) {
|
|
|
|
limit--;
|
|
|
|
}
|
|
|
|
return Slice(s.data() + start, limit - start);
|
|
|
|
}
|
|
|
|
#endif
|
|
|
|
|
2011-03-22 19:32:49 +01:00
|
|
|
void PrintEnvironment() {
|
|
|
|
fprintf(stderr, "LevelDB: version %d.%d\n",
|
|
|
|
kMajorVersion, kMinorVersion);
|
|
|
|
|
|
|
|
#if defined(__linux)
|
2013-02-15 20:53:17 +01:00
|
|
|
time_t now = time(nullptr);
|
2011-03-22 19:32:49 +01:00
|
|
|
fprintf(stderr, "Date: %s", ctime(&now)); // ctime() adds newline
|
|
|
|
|
|
|
|
FILE* cpuinfo = fopen("/proc/cpuinfo", "r");
|
2013-02-15 20:53:17 +01:00
|
|
|
if (cpuinfo != nullptr) {
|
2011-03-22 19:32:49 +01:00
|
|
|
char line[1000];
|
|
|
|
int num_cpus = 0;
|
|
|
|
std::string cpu_type;
|
|
|
|
std::string cache_size;
|
2013-02-15 20:53:17 +01:00
|
|
|
while (fgets(line, sizeof(line), cpuinfo) != nullptr) {
|
2011-03-22 19:32:49 +01:00
|
|
|
const char* sep = strchr(line, ':');
|
2013-02-15 20:53:17 +01:00
|
|
|
if (sep == nullptr) {
|
2011-03-22 19:32:49 +01:00
|
|
|
continue;
|
|
|
|
}
|
|
|
|
Slice key = TrimSpace(Slice(line, sep - 1 - line));
|
|
|
|
Slice val = TrimSpace(Slice(sep + 1));
|
|
|
|
if (key == "model name") {
|
|
|
|
++num_cpus;
|
|
|
|
cpu_type = val.ToString();
|
|
|
|
} else if (key == "cache size") {
|
|
|
|
cache_size = val.ToString();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
fclose(cpuinfo);
|
|
|
|
fprintf(stderr, "CPU: %d * %s\n", num_cpus, cpu_type.c_str());
|
|
|
|
fprintf(stderr, "CPUCache: %s\n", cache_size.c_str());
|
|
|
|
}
|
|
|
|
#endif
|
|
|
|
}
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
public:
|
2011-04-12 21:38:58 +02:00
|
|
|
Benchmark()
|
2015-03-07 01:38:50 +01:00
|
|
|
: cache_(
|
|
|
|
FLAGS_cache_size >= 0
|
|
|
|
? (FLAGS_cache_numshardbits >= 1
|
2015-03-17 23:04:37 +01:00
|
|
|
? NewLRUCache(FLAGS_cache_size, FLAGS_cache_numshardbits)
|
2015-03-07 01:38:50 +01:00
|
|
|
: NewLRUCache(FLAGS_cache_size))
|
|
|
|
: nullptr),
|
|
|
|
compressed_cache_(FLAGS_compressed_cache_size >= 0
|
|
|
|
? (FLAGS_cache_numshardbits >= 1
|
|
|
|
? NewLRUCache(FLAGS_compressed_cache_size,
|
|
|
|
FLAGS_cache_numshardbits)
|
|
|
|
: NewLRUCache(FLAGS_compressed_cache_size))
|
|
|
|
: nullptr),
|
|
|
|
filter_policy_(FLAGS_bloom_bits >= 0
|
|
|
|
? NewBloomFilterPolicy(FLAGS_bloom_bits,
|
|
|
|
FLAGS_use_block_based_filter)
|
|
|
|
: nullptr),
|
|
|
|
prefix_extractor_(NewFixedPrefixTransform(FLAGS_prefix_size)),
|
|
|
|
num_(FLAGS_num),
|
|
|
|
value_size_(FLAGS_value_size),
|
|
|
|
key_size_(FLAGS_key_size),
|
|
|
|
prefix_size_(FLAGS_prefix_size),
|
|
|
|
keys_per_prefix_(FLAGS_keys_per_prefix),
|
|
|
|
entries_per_batch_(1),
|
|
|
|
reads_(FLAGS_reads < 0 ? FLAGS_num : FLAGS_reads),
|
|
|
|
read_random_exp_range_(0.0),
|
|
|
|
writes_(FLAGS_writes < 0 ? FLAGS_num : FLAGS_writes),
|
|
|
|
readwrites_(
|
|
|
|
(FLAGS_writes < 0 && FLAGS_reads < 0)
|
|
|
|
? FLAGS_num
|
|
|
|
: ((FLAGS_writes > FLAGS_reads) ? FLAGS_writes : FLAGS_reads)),
|
|
|
|
merge_keys_(FLAGS_merge_keys < 0 ? FLAGS_num : FLAGS_merge_keys),
|
2015-04-10 00:51:34 +02:00
|
|
|
report_file_operations_(FLAGS_report_file_operations),
|
|
|
|
cachedev_fd_(-1) {
|
2014-10-29 22:24:34 +01:00
|
|
|
if (report_file_operations_) {
|
|
|
|
if (!FLAGS_hdfs.empty()) {
|
|
|
|
fprintf(stderr,
|
|
|
|
"--hdfs and --report_file_operations cannot be enabled "
|
|
|
|
"at the same time");
|
|
|
|
exit(1);
|
|
|
|
}
|
|
|
|
FLAGS_env = new ReportFileOpEnv(rocksdb::Env::Default());
|
|
|
|
}
|
|
|
|
|
2014-03-05 02:08:05 +01:00
|
|
|
if (FLAGS_prefix_size > FLAGS_key_size) {
|
|
|
|
fprintf(stderr, "prefix size is larger than key size");
|
|
|
|
exit(1);
|
|
|
|
}
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
std::vector<std::string> files;
|
2012-06-08 10:11:14 +02:00
|
|
|
FLAGS_env->GetChildren(FLAGS_db, &files);
|
2012-11-06 21:02:18 +01:00
|
|
|
for (unsigned int i = 0; i < files.size(); i++) {
|
2011-03-18 23:37:00 +01:00
|
|
|
if (Slice(files[i]).starts_with("heap-")) {
|
2013-10-24 16:43:14 +02:00
|
|
|
FLAGS_env->DeleteFile(FLAGS_db + "/" + files[i]);
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
}
|
2011-05-21 04:17:43 +02:00
|
|
|
if (!FLAGS_use_existing_db) {
|
benchmark.sh won't run through all tests properly if one specifies wal_dir to be different than db directory.
Summary:
A command line like this to run all the tests:
source benchmark.config.sh && nohup ./benchmark.sh 'bulkload,fillseq,overwrite,filluniquerandom,readrandom,readwhilewriting'
where
benchmark.config.sh is:
export DB_DIR=/data/mysql/rocksdata
export WAL_DIR=/txlogs/rockswal
export OUTPUT_DIR=/root/rocks_benchmarking/output
Will fail for the tests that need a new DB .
Also 1) set disable_data_sync=0 and 2) add debug mode to run through all the tests more quickly
Test Plan: run ./benchmark.sh 'debug,bulkload,fillseq,overwrite,filluniquerandom,readrandom,readwhilewriting' and verify that there are no complaints about WAL dir not being empty.
Reviewers: sdong, yhchiang, rven, igor
Reviewed By: igor
Subscribers: dhruba
Differential Revision: https://reviews.facebook.net/D30909
2015-01-06 00:36:47 +01:00
|
|
|
Options options;
|
|
|
|
if (!FLAGS_wal_dir.empty()) {
|
|
|
|
options.wal_dir = FLAGS_wal_dir;
|
|
|
|
}
|
|
|
|
DestroyDB(FLAGS_db, options);
|
2011-05-21 04:17:43 +02:00
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
~Benchmark() {
|
2015-05-29 23:36:35 +02:00
|
|
|
db_.DeleteDBs();
|
2013-08-23 23:49:57 +02:00
|
|
|
delete prefix_extractor_;
|
2015-03-14 00:41:00 +01:00
|
|
|
if (cache_.get() != nullptr) {
|
|
|
|
// this will leak, but we're shutting down so nobody cares
|
|
|
|
cache_->DisownData();
|
|
|
|
}
|
2015-04-03 03:14:49 +02:00
|
|
|
if (FLAGS_disable_flashcache_for_background_threads && cachedev_fd_ != -1) {
|
2015-04-10 00:51:34 +02:00
|
|
|
// Dtor for this env should run before cachedev_fd_ is closed
|
|
|
|
flashcache_aware_env_ = nullptr;
|
2015-04-03 03:14:49 +02:00
|
|
|
close(cachedev_fd_);
|
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2015-02-19 23:27:48 +01:00
|
|
|
Slice AllocateKey(std::unique_ptr<const char[]>* key_guard) {
|
2015-03-26 09:53:42 +01:00
|
|
|
char* data = new char[key_size_];
|
|
|
|
const char* const_data = data;
|
|
|
|
key_guard->reset(const_data);
|
2015-02-19 23:27:48 +01:00
|
|
|
return Slice(key_guard->get(), key_size_);
|
2014-04-08 20:21:09 +02:00
|
|
|
}
|
|
|
|
|
2014-03-05 02:08:05 +01:00
|
|
|
// Generate key according to the given specification and random number.
|
|
|
|
// The resulting key will have the following format (if keys_per_prefix_
|
|
|
|
// is positive), extra trailing bytes are either cut off or paddd with '0'.
|
|
|
|
// The prefix value is derived from key value.
|
|
|
|
// ----------------------------
|
|
|
|
// | prefix 00000 | key 00000 |
|
|
|
|
// ----------------------------
|
|
|
|
// If keys_per_prefix_ is 0, the key is simply a binary representation of
|
|
|
|
// random number followed by trailing '0's
|
|
|
|
// ----------------------------
|
|
|
|
// | key 00000 |
|
|
|
|
// ----------------------------
|
2014-04-08 20:21:09 +02:00
|
|
|
void GenerateKeyFromInt(uint64_t v, int64_t num_keys, Slice* key) {
|
|
|
|
char* start = const_cast<char*>(key->data());
|
2014-03-05 02:08:05 +01:00
|
|
|
char* pos = start;
|
|
|
|
if (keys_per_prefix_ > 0) {
|
|
|
|
int64_t num_prefix = num_keys / keys_per_prefix_;
|
|
|
|
int64_t prefix = v % num_prefix;
|
|
|
|
int bytes_to_fill = std::min(prefix_size_, 8);
|
|
|
|
if (port::kLittleEndian) {
|
|
|
|
for (int i = 0; i < bytes_to_fill; ++i) {
|
|
|
|
pos[i] = (prefix >> ((bytes_to_fill - i - 1) << 3)) & 0xFF;
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
memcpy(pos, static_cast<void*>(&prefix), bytes_to_fill);
|
|
|
|
}
|
|
|
|
if (prefix_size_ > 8) {
|
|
|
|
// fill the rest with 0s
|
|
|
|
memset(pos + 8, '0', prefix_size_ - 8);
|
|
|
|
}
|
|
|
|
pos += prefix_size_;
|
|
|
|
}
|
|
|
|
|
|
|
|
int bytes_to_fill = std::min(key_size_ - static_cast<int>(pos - start), 8);
|
|
|
|
if (port::kLittleEndian) {
|
|
|
|
for (int i = 0; i < bytes_to_fill; ++i) {
|
|
|
|
pos[i] = (v >> ((bytes_to_fill - i - 1) << 3)) & 0xFF;
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
memcpy(pos, static_cast<void*>(&v), bytes_to_fill);
|
|
|
|
}
|
|
|
|
pos += bytes_to_fill;
|
|
|
|
if (key_size_ > pos - start) {
|
|
|
|
memset(pos, '0', key_size_ - (pos - start));
|
|
|
|
}
|
2013-08-13 22:58:02 +02:00
|
|
|
}
|
|
|
|
|
2014-04-11 21:15:09 +02:00
|
|
|
std::string GetDbNameForMultiple(std::string base_name, size_t id) {
|
2014-11-25 05:44:49 +01:00
|
|
|
return base_name + ToString(id);
|
2014-04-11 21:15:09 +02:00
|
|
|
}
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
void Run() {
|
2014-06-23 19:46:16 +02:00
|
|
|
if (!SanityCheck()) {
|
|
|
|
exit(1);
|
|
|
|
}
|
2011-03-22 19:32:49 +01:00
|
|
|
PrintHeader();
|
2014-12-23 03:39:28 +01:00
|
|
|
Open(&open_options_);
|
2015-08-11 20:46:15 +02:00
|
|
|
std::stringstream benchmark_stream(FLAGS_benchmarks);
|
|
|
|
std::string name;
|
|
|
|
while (std::getline(benchmark_stream, name, ',')) {
|
2013-08-13 22:58:02 +02:00
|
|
|
// Sanitize parameters
|
2011-08-22 23:08:51 +02:00
|
|
|
num_ = FLAGS_num;
|
2011-09-01 21:08:02 +02:00
|
|
|
reads_ = (FLAGS_reads < 0 ? FLAGS_num : FLAGS_reads);
|
2012-05-31 02:04:35 +02:00
|
|
|
writes_ = (FLAGS_writes < 0 ? FLAGS_num : FLAGS_writes);
|
2011-08-22 23:08:51 +02:00
|
|
|
value_size_ = FLAGS_value_size;
|
2013-03-01 23:10:09 +01:00
|
|
|
key_size_ = FLAGS_key_size;
|
2014-10-23 03:43:27 +02:00
|
|
|
entries_per_batch_ = FLAGS_batch_size;
|
2011-08-22 23:08:51 +02:00
|
|
|
write_options_ = WriteOptions();
|
2015-03-07 01:38:50 +01:00
|
|
|
read_random_exp_range_ = FLAGS_read_random_exp_range;
|
2012-05-31 02:04:35 +02:00
|
|
|
if (FLAGS_sync) {
|
|
|
|
write_options_.sync = true;
|
|
|
|
}
|
2012-08-20 01:05:02 +02:00
|
|
|
write_options_.disableWAL = FLAGS_disable_wal;
|
|
|
|
|
2013-02-15 20:53:17 +01:00
|
|
|
void (Benchmark::*method)(ThreadState*) = nullptr;
|
2015-05-29 23:36:35 +02:00
|
|
|
void (Benchmark::*post_process_method)() = nullptr;
|
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
bool fresh_db = false;
|
2011-09-01 21:08:02 +02:00
|
|
|
int num_threads = FLAGS_threads;
|
2011-03-22 19:32:49 +01:00
|
|
|
|
2015-08-11 20:46:15 +02:00
|
|
|
if (name == "fillseq") {
|
2011-08-22 23:08:51 +02:00
|
|
|
fresh_db = true;
|
|
|
|
method = &Benchmark::WriteSeq;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "fillbatch") {
|
2011-08-22 23:08:51 +02:00
|
|
|
fresh_db = true;
|
|
|
|
entries_per_batch_ = 1000;
|
|
|
|
method = &Benchmark::WriteSeq;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "fillrandom") {
|
2011-08-22 23:08:51 +02:00
|
|
|
fresh_db = true;
|
|
|
|
method = &Benchmark::WriteRandom;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "filluniquerandom") {
|
2013-06-15 01:17:56 +02:00
|
|
|
fresh_db = true;
|
|
|
|
if (num_threads > 1) {
|
2015-08-11 20:46:15 +02:00
|
|
|
fprintf(stderr,
|
|
|
|
"filluniquerandom multithreaded not supported"
|
|
|
|
", use 1 thread");
|
2014-04-05 00:37:28 +02:00
|
|
|
num_threads = 1;
|
2013-06-15 01:17:56 +02:00
|
|
|
}
|
|
|
|
method = &Benchmark::WriteUniqueRandom;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "overwrite") {
|
2011-08-22 23:08:51 +02:00
|
|
|
method = &Benchmark::WriteRandom;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "fillsync") {
|
2011-08-22 23:08:51 +02:00
|
|
|
fresh_db = true;
|
|
|
|
num_ /= 1000;
|
|
|
|
write_options_.sync = true;
|
|
|
|
method = &Benchmark::WriteRandom;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "fill100K") {
|
2011-08-22 23:08:51 +02:00
|
|
|
fresh_db = true;
|
|
|
|
num_ /= 1000;
|
|
|
|
value_size_ = 100 * 1000;
|
|
|
|
method = &Benchmark::WriteRandom;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "readseq") {
|
2011-08-22 23:08:51 +02:00
|
|
|
method = &Benchmark::ReadSequential;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "readtocache") {
|
2013-12-19 01:50:48 +01:00
|
|
|
method = &Benchmark::ReadSequential;
|
|
|
|
num_threads = 1;
|
|
|
|
reads_ = num_;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "readreverse") {
|
2011-08-22 23:08:51 +02:00
|
|
|
method = &Benchmark::ReadReverse;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "readrandom") {
|
2011-08-22 23:08:51 +02:00
|
|
|
method = &Benchmark::ReadRandom;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "readrandomfast") {
|
2014-09-25 20:14:01 +02:00
|
|
|
method = &Benchmark::ReadRandomFast;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "multireadrandom") {
|
2014-09-30 01:05:25 +02:00
|
|
|
fprintf(stderr, "entries_per_batch = %" PRIi64 "\n",
|
|
|
|
entries_per_batch_);
|
2014-04-08 20:21:09 +02:00
|
|
|
method = &Benchmark::MultiReadRandom;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "readmissing") {
|
2014-04-08 20:21:09 +02:00
|
|
|
++key_size_;
|
|
|
|
method = &Benchmark::ReadRandom;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "newiterator") {
|
2014-02-03 22:13:36 +01:00
|
|
|
method = &Benchmark::IteratorCreation;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "newiteratorwhilewriting") {
|
2014-04-10 19:15:59 +02:00
|
|
|
num_threads++; // Add extra thread for writing
|
|
|
|
method = &Benchmark::IteratorCreationWhileWriting;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "seekrandom") {
|
2012-04-17 17:36:46 +02:00
|
|
|
method = &Benchmark::SeekRandom;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "seekrandomwhilewriting") {
|
2014-04-11 18:47:20 +02:00
|
|
|
num_threads++; // Add extra thread for writing
|
|
|
|
method = &Benchmark::SeekRandomWhileWriting;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "seekrandomwhilemerging") {
|
2015-03-30 20:28:25 +02:00
|
|
|
num_threads++; // Add extra thread for merging
|
|
|
|
method = &Benchmark::SeekRandomWhileMerging;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "readrandomsmall") {
|
2011-05-21 04:17:43 +02:00
|
|
|
reads_ /= 1000;
|
2011-08-22 23:08:51 +02:00
|
|
|
method = &Benchmark::ReadRandom;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "deleteseq") {
|
2012-04-17 17:36:46 +02:00
|
|
|
method = &Benchmark::DeleteSeq;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "deleterandom") {
|
2012-04-17 17:36:46 +02:00
|
|
|
method = &Benchmark::DeleteRandom;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "readwhilewriting") {
|
2011-09-01 21:08:02 +02:00
|
|
|
num_threads++; // Add extra thread for writing
|
|
|
|
method = &Benchmark::ReadWhileWriting;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "readwhilemerging") {
|
2015-03-18 21:50:52 +01:00
|
|
|
num_threads++; // Add extra thread for writing
|
|
|
|
method = &Benchmark::ReadWhileMerging;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "readrandomwriterandom") {
|
2012-08-31 08:06:17 +02:00
|
|
|
method = &Benchmark::ReadRandomWriteRandom;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "readrandommergerandom") {
|
2014-01-11 02:33:56 +01:00
|
|
|
if (FLAGS_merge_operator.empty()) {
|
|
|
|
fprintf(stdout, "%-12s : skipped (--merge_operator is unknown)\n",
|
2015-08-11 20:46:15 +02:00
|
|
|
name.c_str());
|
2014-04-08 20:21:09 +02:00
|
|
|
exit(1);
|
2014-01-11 02:33:56 +01:00
|
|
|
}
|
2014-04-08 20:21:09 +02:00
|
|
|
method = &Benchmark::ReadRandomMergeRandom;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "updaterandom") {
|
2013-03-01 20:25:13 +01:00
|
|
|
method = &Benchmark::UpdateRandom;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "appendrandom") {
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
method = &Benchmark::AppendRandom;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "mergerandom") {
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
if (FLAGS_merge_operator.empty()) {
|
|
|
|
fprintf(stdout, "%-12s : skipped (--merge_operator is unknown)\n",
|
2015-08-11 20:46:15 +02:00
|
|
|
name.c_str());
|
2014-04-08 20:21:09 +02:00
|
|
|
exit(1);
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
}
|
2014-04-08 20:21:09 +02:00
|
|
|
method = &Benchmark::MergeRandom;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "randomwithverify") {
|
2013-02-20 02:51:06 +01:00
|
|
|
method = &Benchmark::RandomWithVerify;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "fillseekseq") {
|
SkipListRep::LookaheadIterator
Summary:
This diff introduces the `lookahead` argument to `SkipListFactory()`. This is an
optimization for the tailing use case which includes many seeks. E.g. consider
the following operations on a skip list iterator:
Seek(x), Next(), Next(), Seek(x+2), Next(), Seek(x+3), Next(), Next(), ...
If `lookahead` is positive, `SkipListRep` will return an iterator which also
keeps track of the previously visited node. Seek() then first does a linear
search starting from that node (up to `lookahead` steps). As in the tailing
example above, this may require fewer than ~log(n) comparisons as with regular
skip list search.
Test Plan:
Added a new benchmark (`fillseekseq`) which simulates the usage pattern. It
first writes N records (with consecutive keys), then measures how much time it
takes to read them by calling `Seek()` and `Next()`.
$ time ./db_bench -num 10000000 -benchmarks fillseekseq -prefix_size 1 \
-key_size 8 -write_buffer_size $[1024*1024*1024] -value_size 50 \
-seekseq_next 2 -skip_list_lookahead=0
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.389 micros/op 2569047 ops/sec;
real 0m21.806s
user 0m12.106s
sys 0m9.672s
$ time ./db_bench [...] -skip_list_lookahead=2
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.153 micros/op 6540684 ops/sec;
real 0m19.469s
user 0m10.192s
sys 0m9.252s
Reviewers: ljin, sdong, igor
Reviewed By: igor
Subscribers: dhruba, leveldb, march, lovro
Differential Revision: https://reviews.facebook.net/D23997
2014-09-24 00:52:28 +02:00
|
|
|
method = &Benchmark::WriteSeqSeekSeq;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "compact") {
|
2011-08-22 23:08:51 +02:00
|
|
|
method = &Benchmark::Compact;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "crc32c") {
|
2011-08-22 23:08:51 +02:00
|
|
|
method = &Benchmark::Crc32c;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "xxhash") {
|
2014-05-01 20:09:32 +02:00
|
|
|
method = &Benchmark::xxHash;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "acquireload") {
|
2011-08-22 23:08:51 +02:00
|
|
|
method = &Benchmark::AcquireLoad;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "compress") {
|
2014-02-08 03:12:30 +01:00
|
|
|
method = &Benchmark::Compress;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "uncompress") {
|
2014-02-08 03:12:30 +01:00
|
|
|
method = &Benchmark::Uncompress;
|
2015-10-14 21:43:00 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "randomtransaction") {
|
2015-05-29 23:36:35 +02:00
|
|
|
method = &Benchmark::RandomTransaction;
|
|
|
|
post_process_method = &Benchmark::RandomTransactionVerify;
|
2015-10-14 21:43:00 +02:00
|
|
|
#endif // ROCKSDB_LITE
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
} else if (name == "randomreplacekeys") {
|
|
|
|
fresh_db = true;
|
|
|
|
method = &Benchmark::RandomReplaceKeys;
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "stats") {
|
2013-10-05 07:32:05 +02:00
|
|
|
PrintStats("rocksdb.stats");
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "levelstats") {
|
2013-10-05 07:32:05 +02:00
|
|
|
PrintStats("rocksdb.levelstats");
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (name == "sstables") {
|
2013-10-05 07:32:05 +02:00
|
|
|
PrintStats("rocksdb.sstables");
|
2015-08-11 20:46:15 +02:00
|
|
|
} else if (!name.empty()) { // No error message for empty name
|
|
|
|
fprintf(stderr, "unknown benchmark '%s'\n", name.c_str());
|
|
|
|
exit(1);
|
2011-04-12 21:38:58 +02:00
|
|
|
}
|
2011-08-22 23:08:51 +02:00
|
|
|
|
|
|
|
if (fresh_db) {
|
|
|
|
if (FLAGS_use_existing_db) {
|
|
|
|
fprintf(stdout, "%-12s : skipped (--use_existing_db is true)\n",
|
2015-08-11 20:46:15 +02:00
|
|
|
name.c_str());
|
2013-02-15 20:53:17 +01:00
|
|
|
method = nullptr;
|
2011-08-22 23:08:51 +02:00
|
|
|
} else {
|
2014-08-19 03:15:01 +02:00
|
|
|
if (db_.db != nullptr) {
|
2015-05-29 23:36:35 +02:00
|
|
|
db_.DeleteDBs();
|
2014-12-23 03:39:28 +01:00
|
|
|
DestroyDB(FLAGS_db, open_options_);
|
2014-04-11 21:15:09 +02:00
|
|
|
}
|
|
|
|
for (size_t i = 0; i < multi_dbs_.size(); i++) {
|
2014-08-19 03:15:01 +02:00
|
|
|
delete multi_dbs_[i].db;
|
2014-12-23 03:39:28 +01:00
|
|
|
DestroyDB(GetDbNameForMultiple(FLAGS_db, i), open_options_);
|
2014-04-11 21:15:09 +02:00
|
|
|
}
|
|
|
|
multi_dbs_.clear();
|
2011-08-22 23:08:51 +02:00
|
|
|
}
|
2014-12-23 03:39:28 +01:00
|
|
|
Open(&open_options_); // use open_options for the last accessed
|
2011-08-22 23:08:51 +02:00
|
|
|
}
|
|
|
|
|
2013-02-15 20:53:17 +01:00
|
|
|
if (method != nullptr) {
|
2013-10-24 16:43:14 +02:00
|
|
|
fprintf(stdout, "DB path: [%s]\n", FLAGS_db.c_str());
|
2011-09-01 21:08:02 +02:00
|
|
|
RunBenchmark(num_threads, name, method);
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
2015-05-29 23:36:35 +02:00
|
|
|
if (post_process_method != nullptr) {
|
|
|
|
(this->*post_process_method)();
|
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
2013-06-19 05:28:41 +02:00
|
|
|
if (FLAGS_statistics) {
|
|
|
|
fprintf(stdout, "STATISTICS:\n%s\n", dbstats->ToString().c_str());
|
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2011-03-22 19:32:49 +01:00
|
|
|
private:
|
2015-03-30 18:51:11 +02:00
|
|
|
std::unique_ptr<Env> flashcache_aware_env_;
|
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
struct ThreadArg {
|
|
|
|
Benchmark* bm;
|
|
|
|
SharedState* shared;
|
|
|
|
ThreadState* thread;
|
|
|
|
void (Benchmark::*method)(ThreadState*);
|
|
|
|
};
|
|
|
|
|
|
|
|
static void ThreadBody(void* v) {
|
|
|
|
ThreadArg* arg = reinterpret_cast<ThreadArg*>(v);
|
|
|
|
SharedState* shared = arg->shared;
|
|
|
|
ThreadState* thread = arg->thread;
|
|
|
|
{
|
|
|
|
MutexLock l(&shared->mu);
|
|
|
|
shared->num_initialized++;
|
|
|
|
if (shared->num_initialized >= shared->total) {
|
|
|
|
shared->cv.SignalAll();
|
|
|
|
}
|
|
|
|
while (!shared->start) {
|
|
|
|
shared->cv.Wait();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2013-11-18 20:32:54 +01:00
|
|
|
SetPerfLevel(static_cast<PerfLevel> (shared->perf_level));
|
2012-10-03 15:51:11 +02:00
|
|
|
thread->stats.Start(thread->tid);
|
2011-08-22 23:08:51 +02:00
|
|
|
(arg->bm->*(arg->method))(thread);
|
|
|
|
thread->stats.Stop();
|
|
|
|
|
|
|
|
{
|
|
|
|
MutexLock l(&shared->mu);
|
|
|
|
shared->num_done++;
|
|
|
|
if (shared->num_done >= shared->total) {
|
|
|
|
shared->cv.SignalAll();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2011-09-01 21:08:02 +02:00
|
|
|
void RunBenchmark(int n, Slice name,
|
|
|
|
void (Benchmark::*method)(ThreadState*)) {
|
2011-08-22 23:08:51 +02:00
|
|
|
SharedState shared;
|
|
|
|
shared.total = n;
|
|
|
|
shared.num_initialized = 0;
|
|
|
|
shared.num_done = 0;
|
|
|
|
shared.start = false;
|
2015-06-18 01:44:52 +02:00
|
|
|
if (FLAGS_benchmark_write_rate_limit > 0) {
|
|
|
|
shared.write_rate_limiter.reset(
|
|
|
|
NewGenericRateLimiter(FLAGS_benchmark_write_rate_limit));
|
|
|
|
}
|
2011-08-22 23:08:51 +02:00
|
|
|
|
db_bench periodically writes QPS to CSV file
Summary:
This is part of an effort to better understand and optimize RocksDB stalls under high load. I added a feature to db_bench to periodically write QPS to CSV files. That way we can nicely see how our QPS changes in time (especially when DB is stalled) and can do a better job of evaluating our stall system (i.e. we want the QPS to be as constant as possible, as opposed to having bunch of stalls)
Cool part of CSV files is that we can easily graph them -- there are a bunch of tools available.
Test Plan:
Ran ./db_bench --report_interval_seconds=10 --benchmarks=fillrandom --num=10000000
and observed this in report.csv:
secs_elapsed,interval_qps
10,2725860
20,1980480
30,1863456
40,1454359
50,1460389
Reviewers: sdong, MarkCallaghan, rven, yhchiang
Reviewed By: yhchiang
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D40047
2015-06-12 23:31:53 +02:00
|
|
|
std::unique_ptr<ReporterAgent> reporter_agent;
|
|
|
|
if (FLAGS_report_interval_seconds > 0) {
|
|
|
|
reporter_agent.reset(new ReporterAgent(FLAGS_env, FLAGS_report_file,
|
|
|
|
FLAGS_report_interval_seconds));
|
|
|
|
}
|
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
ThreadArg* arg = new ThreadArg[n];
|
2014-07-07 19:53:31 +02:00
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
for (int i = 0; i < n; i++) {
|
2014-07-07 19:53:31 +02:00
|
|
|
#ifdef NUMA
|
|
|
|
if (FLAGS_enable_numa) {
|
|
|
|
// Performs a local allocation of memory to threads in numa node.
|
|
|
|
int n_nodes = numa_num_task_nodes(); // Number of nodes in NUMA.
|
|
|
|
numa_exit_on_error = 1;
|
|
|
|
int numa_node = i % n_nodes;
|
|
|
|
bitmask* nodes = numa_allocate_nodemask();
|
|
|
|
numa_bitmask_clearall(nodes);
|
|
|
|
numa_bitmask_setbit(nodes, numa_node);
|
|
|
|
// numa_bind() call binds the process to the node and these
|
|
|
|
// properties are passed on to the thread that is created in
|
|
|
|
// StartThread method called later in the loop.
|
|
|
|
numa_bind(nodes);
|
|
|
|
numa_set_strict(1);
|
|
|
|
numa_free_nodemask(nodes);
|
|
|
|
}
|
|
|
|
#endif
|
2011-08-22 23:08:51 +02:00
|
|
|
arg[i].bm = this;
|
|
|
|
arg[i].method = method;
|
|
|
|
arg[i].shared = &shared;
|
|
|
|
arg[i].thread = new ThreadState(i);
|
db_bench periodically writes QPS to CSV file
Summary:
This is part of an effort to better understand and optimize RocksDB stalls under high load. I added a feature to db_bench to periodically write QPS to CSV files. That way we can nicely see how our QPS changes in time (especially when DB is stalled) and can do a better job of evaluating our stall system (i.e. we want the QPS to be as constant as possible, as opposed to having bunch of stalls)
Cool part of CSV files is that we can easily graph them -- there are a bunch of tools available.
Test Plan:
Ran ./db_bench --report_interval_seconds=10 --benchmarks=fillrandom --num=10000000
and observed this in report.csv:
secs_elapsed,interval_qps
10,2725860
20,1980480
30,1863456
40,1454359
50,1460389
Reviewers: sdong, MarkCallaghan, rven, yhchiang
Reviewed By: yhchiang
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D40047
2015-06-12 23:31:53 +02:00
|
|
|
arg[i].thread->stats.SetReporterAgent(reporter_agent.get());
|
2011-09-01 21:08:02 +02:00
|
|
|
arg[i].thread->shared = &shared;
|
2012-06-08 10:11:14 +02:00
|
|
|
FLAGS_env->StartThread(ThreadBody, &arg[i]);
|
2011-08-22 23:08:51 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
shared.mu.Lock();
|
|
|
|
while (shared.num_initialized < n) {
|
|
|
|
shared.cv.Wait();
|
|
|
|
}
|
|
|
|
|
|
|
|
shared.start = true;
|
|
|
|
shared.cv.SignalAll();
|
|
|
|
while (shared.num_done < n) {
|
|
|
|
shared.cv.Wait();
|
|
|
|
}
|
|
|
|
shared.mu.Unlock();
|
|
|
|
|
2013-03-19 14:36:03 +01:00
|
|
|
// Stats for some threads can be excluded.
|
|
|
|
Stats merge_stats;
|
|
|
|
for (int i = 0; i < n; i++) {
|
|
|
|
merge_stats.Merge(arg[i].thread->stats);
|
2011-08-22 23:08:51 +02:00
|
|
|
}
|
2013-03-19 14:36:03 +01:00
|
|
|
merge_stats.Report(name);
|
2011-08-22 23:08:51 +02:00
|
|
|
|
|
|
|
for (int i = 0; i < n; i++) {
|
|
|
|
delete arg[i].thread;
|
|
|
|
}
|
|
|
|
delete[] arg;
|
|
|
|
}
|
|
|
|
|
|
|
|
void Crc32c(ThreadState* thread) {
|
2011-03-25 21:27:43 +01:00
|
|
|
// Checksum about 500MB of data total
|
2011-08-22 23:08:51 +02:00
|
|
|
const int size = 4096;
|
|
|
|
const char* label = "(4K per op)";
|
2011-03-28 22:43:44 +02:00
|
|
|
std::string data(size, 'x');
|
2011-03-25 21:27:43 +01:00
|
|
|
int64_t bytes = 0;
|
|
|
|
uint32_t crc = 0;
|
|
|
|
while (bytes < 500 * 1048576) {
|
|
|
|
crc = crc32c::Value(data.data(), size);
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(nullptr, nullptr, 1);
|
2011-03-25 21:27:43 +01:00
|
|
|
bytes += size;
|
|
|
|
}
|
|
|
|
// Print so result is not dead
|
|
|
|
fprintf(stderr, "... crc=0x%x\r", static_cast<unsigned int>(crc));
|
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
thread->stats.AddBytes(bytes);
|
|
|
|
thread->stats.AddMessage(label);
|
2011-03-25 21:27:43 +01:00
|
|
|
}
|
|
|
|
|
2014-05-01 20:09:32 +02:00
|
|
|
void xxHash(ThreadState* thread) {
|
|
|
|
// Checksum about 500MB of data total
|
|
|
|
const int size = 4096;
|
|
|
|
const char* label = "(4K per op)";
|
|
|
|
std::string data(size, 'x');
|
|
|
|
int64_t bytes = 0;
|
|
|
|
unsigned int xxh32 = 0;
|
|
|
|
while (bytes < 500 * 1048576) {
|
|
|
|
xxh32 = XXH32(data.data(), size, 0);
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(nullptr, nullptr, 1);
|
2014-05-01 20:09:32 +02:00
|
|
|
bytes += size;
|
|
|
|
}
|
|
|
|
// Print so result is not dead
|
|
|
|
fprintf(stderr, "... xxh32=0x%x\r", static_cast<unsigned int>(xxh32));
|
|
|
|
|
|
|
|
thread->stats.AddBytes(bytes);
|
|
|
|
thread->stats.AddMessage(label);
|
|
|
|
}
|
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
void AcquireLoad(ThreadState* thread) {
|
2011-05-28 02:53:58 +02:00
|
|
|
int dummy;
|
2014-10-27 23:41:05 +01:00
|
|
|
std::atomic<void*> ap(&dummy);
|
2011-05-28 02:53:58 +02:00
|
|
|
int count = 0;
|
2013-02-15 20:53:17 +01:00
|
|
|
void *ptr = nullptr;
|
2011-08-22 23:08:51 +02:00
|
|
|
thread->stats.AddMessage("(each op is 1000 loads)");
|
2011-05-28 02:53:58 +02:00
|
|
|
while (count < 100000) {
|
|
|
|
for (int i = 0; i < 1000; i++) {
|
2014-10-27 22:50:21 +01:00
|
|
|
ptr = ap.load(std::memory_order_acquire);
|
2011-05-28 02:53:58 +02:00
|
|
|
}
|
|
|
|
count++;
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(nullptr, nullptr, 1);
|
2011-05-28 02:53:58 +02:00
|
|
|
}
|
2013-02-15 20:53:17 +01:00
|
|
|
if (ptr == nullptr) exit(1); // Disable unused variable warning.
|
2011-05-28 02:53:58 +02:00
|
|
|
}
|
|
|
|
|
2014-02-08 03:12:30 +01:00
|
|
|
void Compress(ThreadState *thread) {
|
2011-08-22 23:08:51 +02:00
|
|
|
RandomGenerator gen;
|
2014-08-25 23:22:05 +02:00
|
|
|
Slice input = gen.Generate(FLAGS_block_size);
|
2011-04-12 21:38:58 +02:00
|
|
|
int64_t bytes = 0;
|
|
|
|
int64_t produced = 0;
|
|
|
|
bool ok = true;
|
|
|
|
std::string compressed;
|
2014-02-08 03:12:30 +01:00
|
|
|
|
|
|
|
// Compress 1G
|
|
|
|
while (ok && bytes < int64_t(1) << 30) {
|
2015-08-11 20:46:15 +02:00
|
|
|
ok = CompressSlice(input, &compressed);
|
2011-04-12 21:38:58 +02:00
|
|
|
produced += compressed.size();
|
|
|
|
bytes += input.size();
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(nullptr, nullptr, 1);
|
2011-04-12 21:38:58 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
if (!ok) {
|
2014-02-08 03:12:30 +01:00
|
|
|
thread->stats.AddMessage("(compression failure)");
|
2011-04-12 21:38:58 +02:00
|
|
|
} else {
|
|
|
|
char buf[100];
|
|
|
|
snprintf(buf, sizeof(buf), "(output: %.1f%%)",
|
|
|
|
(produced * 100.0) / bytes);
|
2011-08-22 23:08:51 +02:00
|
|
|
thread->stats.AddMessage(buf);
|
|
|
|
thread->stats.AddBytes(bytes);
|
2011-04-12 21:38:58 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-02-08 03:12:30 +01:00
|
|
|
void Uncompress(ThreadState *thread) {
|
2011-08-22 23:08:51 +02:00
|
|
|
RandomGenerator gen;
|
2014-08-25 23:22:05 +02:00
|
|
|
Slice input = gen.Generate(FLAGS_block_size);
|
2011-04-12 21:38:58 +02:00
|
|
|
std::string compressed;
|
2014-02-08 03:12:30 +01:00
|
|
|
|
2015-08-11 20:46:15 +02:00
|
|
|
bool ok = CompressSlice(input, &compressed);
|
2011-04-12 21:38:58 +02:00
|
|
|
int64_t bytes = 0;
|
2014-02-08 03:12:30 +01:00
|
|
|
int decompress_size;
|
|
|
|
while (ok && bytes < 1024 * 1048576) {
|
|
|
|
char *uncompressed = nullptr;
|
|
|
|
switch (FLAGS_compression_type_e) {
|
|
|
|
case rocksdb::kSnappyCompression:
|
|
|
|
// allocate here to make comparison fair
|
|
|
|
uncompressed = new char[input.size()];
|
2015-01-09 22:04:06 +01:00
|
|
|
ok = Snappy_Uncompress(compressed.data(), compressed.size(),
|
|
|
|
uncompressed);
|
2014-02-08 03:12:30 +01:00
|
|
|
break;
|
|
|
|
case rocksdb::kZlibCompression:
|
2015-01-09 22:04:06 +01:00
|
|
|
uncompressed = Zlib_Uncompress(compressed.data(), compressed.size(),
|
2015-01-15 01:24:24 +01:00
|
|
|
&decompress_size, 2);
|
2014-02-08 03:12:30 +01:00
|
|
|
ok = uncompressed != nullptr;
|
|
|
|
break;
|
|
|
|
case rocksdb::kBZip2Compression:
|
2015-01-09 22:04:06 +01:00
|
|
|
uncompressed = BZip2_Uncompress(compressed.data(), compressed.size(),
|
2015-01-15 01:24:24 +01:00
|
|
|
&decompress_size, 2);
|
2014-02-08 03:12:30 +01:00
|
|
|
ok = uncompressed != nullptr;
|
|
|
|
break;
|
|
|
|
case rocksdb::kLZ4Compression:
|
2015-01-09 22:04:06 +01:00
|
|
|
uncompressed = LZ4_Uncompress(compressed.data(), compressed.size(),
|
2015-01-15 01:24:24 +01:00
|
|
|
&decompress_size, 2);
|
2014-02-08 03:12:30 +01:00
|
|
|
ok = uncompressed != nullptr;
|
|
|
|
break;
|
|
|
|
case rocksdb::kLZ4HCCompression:
|
2015-01-09 22:04:06 +01:00
|
|
|
uncompressed = LZ4_Uncompress(compressed.data(), compressed.size(),
|
2015-01-15 01:24:24 +01:00
|
|
|
&decompress_size, 2);
|
2014-02-08 03:12:30 +01:00
|
|
|
ok = uncompressed != nullptr;
|
|
|
|
break;
|
2015-08-28 00:40:42 +02:00
|
|
|
case rocksdb::kZSTDNotFinalCompression:
|
|
|
|
uncompressed = ZSTD_Uncompress(compressed.data(), compressed.size(),
|
|
|
|
&decompress_size);
|
|
|
|
ok = uncompressed != nullptr;
|
|
|
|
break;
|
2014-02-08 03:12:30 +01:00
|
|
|
default:
|
|
|
|
ok = false;
|
|
|
|
}
|
|
|
|
delete[] uncompressed;
|
2011-07-21 04:40:18 +02:00
|
|
|
bytes += input.size();
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(nullptr, nullptr, 1);
|
2011-04-12 21:38:58 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
if (!ok) {
|
2014-02-08 03:12:30 +01:00
|
|
|
thread->stats.AddMessage("(compression failure)");
|
2011-04-12 21:38:58 +02:00
|
|
|
} else {
|
2011-08-22 23:08:51 +02:00
|
|
|
thread->stats.AddBytes(bytes);
|
2011-04-12 21:38:58 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-12-08 21:53:24 +01:00
|
|
|
void Open(Options* opts) {
|
|
|
|
Options& options = *opts;
|
|
|
|
|
2014-08-19 03:15:01 +02:00
|
|
|
assert(db_.db == nullptr);
|
2014-12-08 21:53:24 +01:00
|
|
|
|
2011-05-21 04:17:43 +02:00
|
|
|
options.create_if_missing = !FLAGS_use_existing_db;
|
2014-08-19 03:15:01 +02:00
|
|
|
options.create_missing_column_families = FLAGS_num_column_families > 1;
|
2014-12-02 21:09:20 +01:00
|
|
|
options.db_write_buffer_size = FLAGS_db_write_buffer_size;
|
2011-03-22 19:32:49 +01:00
|
|
|
options.write_buffer_size = FLAGS_write_buffer_size;
|
2012-10-19 23:00:53 +02:00
|
|
|
options.max_write_buffer_number = FLAGS_max_write_buffer_number;
|
2013-06-11 23:23:58 +02:00
|
|
|
options.min_write_buffer_number_to_merge =
|
|
|
|
FLAGS_min_write_buffer_number_to_merge;
|
Support saving history in memtable_list
Summary:
For transactions, we are using the memtables to validate that there are no write conflicts. But after flushing, we don't have any memtables, and transactions could fail to commit. So we want to someone keep around some extra history to use for conflict checking. In addition, we want to provide a way to increase the size of this history if too many transactions fail to commit.
After chatting with people, it seems like everyone prefers just using Memtables to store this history (instead of a separate history structure). It seems like the best place for this is abstracted inside the memtable_list. I decide to create a separate list in MemtableListVersion as using the same list complicated the flush/installalflushresults logic too much.
This diff adds a new parameter to control how much memtable history to keep around after flushing. However, it sounds like people aren't too fond of adding new parameters. So I am making the default size of flushed+not-flushed memtables be set to max_write_buffers. This should not change the maximum amount of memory used, but make it more likely we're using closer the the limit. (We are now postponing deleting flushed memtables until the max_write_buffer limit is reached). So while we might use more memory on average, we are still obeying the limit set (and you could argue it's better to go ahead and use up memory now instead of waiting for a write stall to happen to test this limit).
However, if people are opposed to this default behavior, we can easily set it to 0 and require this parameter be set in order to use transactions.
Test Plan: Added a xfunc test to play around with setting different values of this parameter in all tests. Added testing in memtablelist_test and planning on adding more testing here.
Reviewers: sdong, rven, igor
Reviewed By: igor
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D37443
2015-05-29 01:34:24 +02:00
|
|
|
options.max_write_buffer_number_to_maintain =
|
|
|
|
FLAGS_max_write_buffer_number_to_maintain;
|
2012-10-19 23:00:53 +02:00
|
|
|
options.max_background_compactions = FLAGS_max_background_compactions;
|
2015-08-21 23:25:34 +02:00
|
|
|
options.max_subcompactions = static_cast<uint32_t>(FLAGS_subcompactions);
|
2014-03-05 19:27:17 +01:00
|
|
|
options.max_background_flushes = FLAGS_max_background_flushes;
|
2013-10-24 16:43:14 +02:00
|
|
|
options.compaction_style = FLAGS_compaction_style_e;
|
2015-09-22 02:16:31 +02:00
|
|
|
options.compaction_pri = FLAGS_compaction_pri_e;
|
2014-07-17 02:32:30 +02:00
|
|
|
if (FLAGS_prefix_size != 0) {
|
2014-03-10 20:56:46 +01:00
|
|
|
options.prefix_extractor.reset(
|
|
|
|
NewFixedPrefixTransform(FLAGS_prefix_size));
|
|
|
|
}
|
2014-08-27 19:39:31 +02:00
|
|
|
if (FLAGS_use_uint64_comparator) {
|
|
|
|
options.comparator = test::Uint64Comparator();
|
|
|
|
if (FLAGS_key_size != 8) {
|
|
|
|
fprintf(stderr, "Using Uint64 comparator but key size is not 8.\n");
|
|
|
|
exit(1);
|
|
|
|
}
|
|
|
|
}
|
2014-03-05 19:27:17 +01:00
|
|
|
options.memtable_prefix_bloom_bits = FLAGS_memtable_bloom_bits;
|
2014-03-28 17:21:20 +01:00
|
|
|
options.bloom_locality = FLAGS_bloom_locality;
|
2012-05-30 08:18:16 +02:00
|
|
|
options.max_open_files = FLAGS_open_files;
|
2015-09-30 18:51:31 +02:00
|
|
|
options.max_file_opening_threads = FLAGS_file_opening_threads;
|
2015-08-27 00:25:59 +02:00
|
|
|
options.new_table_reader_for_compaction_inputs =
|
|
|
|
FLAGS_new_table_reader_for_compaction_inputs;
|
|
|
|
options.compaction_readahead_size = FLAGS_compaction_readahead_size;
|
2015-10-27 22:44:16 +01:00
|
|
|
options.random_access_max_buffer_size = FLAGS_random_access_max_buffer_size;
|
2015-10-30 06:10:25 +01:00
|
|
|
options.writable_file_max_buffer_size = FLAGS_writable_file_max_buffer_size;
|
2012-05-30 08:18:16 +02:00
|
|
|
options.statistics = dbstats;
|
2014-08-14 05:49:58 +02:00
|
|
|
if (FLAGS_enable_io_prio) {
|
|
|
|
FLAGS_env->LowerThreadPoolIOPriority(Env::LOW);
|
|
|
|
FLAGS_env->LowerThreadPoolIOPriority(Env::HIGH);
|
|
|
|
}
|
2015-04-10 00:51:34 +02:00
|
|
|
if (FLAGS_disable_flashcache_for_background_threads &&
|
|
|
|
cachedev_fd_ == -1) {
|
|
|
|
// Avoid creating the env twice when an use_existing_db is true
|
2015-04-03 03:14:49 +02:00
|
|
|
cachedev_fd_ = open(FLAGS_flashcache_dev.c_str(), O_RDONLY);
|
|
|
|
if (cachedev_fd_ < 0) {
|
|
|
|
fprintf(stderr, "Open flash device failed\n");
|
|
|
|
exit(1);
|
|
|
|
}
|
2015-03-30 18:51:11 +02:00
|
|
|
flashcache_aware_env_ =
|
2015-04-03 03:14:49 +02:00
|
|
|
std::move(NewFlashcacheAwareEnv(FLAGS_env, cachedev_fd_));
|
2015-03-30 18:51:11 +02:00
|
|
|
if (flashcache_aware_env_.get() == nullptr) {
|
2015-08-11 20:46:15 +02:00
|
|
|
fprintf(stderr, "Failed to open flashcache device at %s\n",
|
2015-03-30 18:51:11 +02:00
|
|
|
FLAGS_flashcache_dev.c_str());
|
|
|
|
std::abort();
|
|
|
|
}
|
|
|
|
options.env = flashcache_aware_env_.get();
|
|
|
|
} else {
|
|
|
|
options.env = FLAGS_env;
|
|
|
|
}
|
2012-08-16 00:44:23 +02:00
|
|
|
options.disableDataSync = FLAGS_disable_data_sync;
|
2012-08-27 21:10:26 +02:00
|
|
|
options.use_fsync = FLAGS_use_fsync;
|
2014-02-25 19:43:46 +01:00
|
|
|
options.wal_dir = FLAGS_wal_dir;
|
2012-10-15 19:10:56 +02:00
|
|
|
options.num_levels = FLAGS_num_levels;
|
2012-08-16 00:44:23 +02:00
|
|
|
options.target_file_size_base = FLAGS_target_file_size_base;
|
|
|
|
options.target_file_size_multiplier = FLAGS_target_file_size_multiplier;
|
|
|
|
options.max_bytes_for_level_base = FLAGS_max_bytes_for_level_base;
|
options.level_compaction_dynamic_level_bytes to allow RocksDB to pick size bases of levels dynamically.
Summary:
When having fixed max_bytes_for_level_base, the ratio of size of largest level and the second one can range from 0 to the multiplier. This makes LSM tree frequently irregular and unpredictable. It can also cause poor space amplification in some cases.
In this improvement (proposed by Igor Kabiljo), we introduce a parameter option.level_compaction_use_dynamic_max_bytes. When turning it on, RocksDB is free to pick a level base in the range of (options.max_bytes_for_level_base/options.max_bytes_for_level_multiplier, options.max_bytes_for_level_base] so that real level ratios are close to options.max_bytes_for_level_multiplier.
Test Plan: New unit tests and pass tests suites including valgrind.
Reviewers: MarkCallaghan, rven, yhchiang, igor, ikabiljo
Reviewed By: ikabiljo
Subscribers: yoshinorim, ikabiljo, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D31437
2015-02-05 20:44:17 +01:00
|
|
|
options.level_compaction_dynamic_level_bytes =
|
|
|
|
FLAGS_level_compaction_dynamic_level_bytes;
|
2012-08-16 00:44:23 +02:00
|
|
|
options.max_bytes_for_level_multiplier =
|
|
|
|
FLAGS_max_bytes_for_level_multiplier;
|
2013-07-13 01:56:52 +02:00
|
|
|
options.filter_deletes = FLAGS_filter_deletes;
|
2015-06-23 19:25:45 +02:00
|
|
|
if (FLAGS_row_cache_size) {
|
|
|
|
if (FLAGS_cache_numshardbits >= 1) {
|
|
|
|
options.row_cache =
|
|
|
|
NewLRUCache(FLAGS_row_cache_size, FLAGS_cache_numshardbits);
|
|
|
|
} else {
|
|
|
|
options.row_cache = NewLRUCache(FLAGS_row_cache_size);
|
|
|
|
}
|
|
|
|
}
|
2014-03-05 19:27:17 +01:00
|
|
|
if ((FLAGS_prefix_size == 0) && (FLAGS_rep_factory == kPrefixHash ||
|
|
|
|
FLAGS_rep_factory == kHashLinkedList)) {
|
|
|
|
fprintf(stderr, "prefix_size should be non-zero if PrefixHash or "
|
|
|
|
"HashLinkedList memtablerep is used\n");
|
2013-08-23 08:10:02 +02:00
|
|
|
exit(1);
|
|
|
|
}
|
|
|
|
switch (FLAGS_rep_factory) {
|
|
|
|
case kSkipList:
|
SkipListRep::LookaheadIterator
Summary:
This diff introduces the `lookahead` argument to `SkipListFactory()`. This is an
optimization for the tailing use case which includes many seeks. E.g. consider
the following operations on a skip list iterator:
Seek(x), Next(), Next(), Seek(x+2), Next(), Seek(x+3), Next(), Next(), ...
If `lookahead` is positive, `SkipListRep` will return an iterator which also
keeps track of the previously visited node. Seek() then first does a linear
search starting from that node (up to `lookahead` steps). As in the tailing
example above, this may require fewer than ~log(n) comparisons as with regular
skip list search.
Test Plan:
Added a new benchmark (`fillseekseq`) which simulates the usage pattern. It
first writes N records (with consecutive keys), then measures how much time it
takes to read them by calling `Seek()` and `Next()`.
$ time ./db_bench -num 10000000 -benchmarks fillseekseq -prefix_size 1 \
-key_size 8 -write_buffer_size $[1024*1024*1024] -value_size 50 \
-seekseq_next 2 -skip_list_lookahead=0
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.389 micros/op 2569047 ops/sec;
real 0m21.806s
user 0m12.106s
sys 0m9.672s
$ time ./db_bench [...] -skip_list_lookahead=2
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.153 micros/op 6540684 ops/sec;
real 0m19.469s
user 0m10.192s
sys 0m9.252s
Reviewers: ljin, sdong, igor
Reviewed By: igor
Subscribers: dhruba, leveldb, march, lovro
Differential Revision: https://reviews.facebook.net/D23997
2014-09-24 00:52:28 +02:00
|
|
|
options.memtable_factory.reset(new SkipListFactory(
|
|
|
|
FLAGS_skip_list_lookahead));
|
2013-08-23 08:10:02 +02:00
|
|
|
break;
|
2014-11-12 22:05:12 +01:00
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
case kPrefixHash:
|
|
|
|
options.memtable_factory.reset(
|
|
|
|
NewHashSkipListRepFactory(FLAGS_hash_bucket_count));
|
|
|
|
break;
|
2014-03-05 19:27:17 +01:00
|
|
|
case kHashLinkedList:
|
|
|
|
options.memtable_factory.reset(NewHashLinkListRepFactory(
|
|
|
|
FLAGS_hash_bucket_count));
|
|
|
|
break;
|
2013-08-23 08:10:02 +02:00
|
|
|
case kVectorRep:
|
|
|
|
options.memtable_factory.reset(
|
|
|
|
new VectorRepFactory
|
|
|
|
);
|
|
|
|
break;
|
Add a new mem-table representation based on cuckoo hash.
Summary:
= Major Changes =
* Add a new mem-table representation, HashCuckooRep, which is based cuckoo hash.
Cuckoo hash uses multiple hash functions. This allows each key to have multiple
possible locations in the mem-table.
- Put: When insert a key, it will try to find whether one of its possible
locations is vacant and store the key. If none of its possible
locations are available, then it will kick out a victim key and
store at that location. The kicked-out victim key will then be
stored at a vacant space of its possible locations or kick-out
another victim. In this diff, the kick-out path (known as
cuckoo-path) is found using BFS, which guarantees to be the shortest.
- Get: Simply tries all possible locations of a key --- this guarantees
worst-case constant time complexity.
- Time complexity: O(1) for Get, and average O(1) for Put if the
fullness of the mem-table is below 80%.
- Default using two hash functions, the number of hash functions used
by the cuckoo-hash may dynamically increase if it fails to find a
short-enough kick-out path.
- Currently, HashCuckooRep does not support iteration and snapshots,
as our current main purpose of this is to optimize point access.
= Minor Changes =
* Add IsSnapshotSupported() to DB to indicate whether the current DB
supports snapshots. If it returns false, then DB::GetSnapshot() will
always return nullptr.
Test Plan:
Run existing tests. Will develop a test specifically for cuckoo hash in
the next diff.
Reviewers: sdong, haobo
Reviewed By: sdong
CC: leveldb, dhruba, igor
Differential Revision: https://reviews.facebook.net/D16155
2014-04-30 02:13:46 +02:00
|
|
|
case kCuckoo:
|
|
|
|
options.memtable_factory.reset(NewHashCuckooRepFactory(
|
|
|
|
options.write_buffer_size, FLAGS_key_size + FLAGS_value_size));
|
|
|
|
break;
|
2014-11-12 22:05:12 +01:00
|
|
|
#else
|
|
|
|
default:
|
|
|
|
fprintf(stderr, "Only skip list is supported in lite mode\n");
|
|
|
|
exit(1);
|
|
|
|
#endif // ROCKSDB_LITE
|
2013-08-23 08:10:02 +02:00
|
|
|
}
|
2014-02-12 19:41:55 +01:00
|
|
|
if (FLAGS_use_plain_table) {
|
2014-11-12 22:05:12 +01:00
|
|
|
#ifndef ROCKSDB_LITE
|
2014-03-05 19:27:17 +01:00
|
|
|
if (FLAGS_rep_factory != kPrefixHash &&
|
|
|
|
FLAGS_rep_factory != kHashLinkedList) {
|
2014-02-12 19:41:55 +01:00
|
|
|
fprintf(stderr, "Waring: plain table is used with skipList\n");
|
|
|
|
}
|
|
|
|
if (!FLAGS_mmap_read && !FLAGS_mmap_write) {
|
|
|
|
fprintf(stderr, "plain table format requires mmap to operate\n");
|
|
|
|
exit(1);
|
|
|
|
}
|
|
|
|
|
|
|
|
int bloom_bits_per_key = FLAGS_bloom_bits;
|
|
|
|
if (bloom_bits_per_key < 0) {
|
|
|
|
bloom_bits_per_key = 0;
|
|
|
|
}
|
2014-07-18 09:08:38 +02:00
|
|
|
|
|
|
|
PlainTableOptions plain_table_options;
|
|
|
|
plain_table_options.user_key_len = FLAGS_key_size;
|
|
|
|
plain_table_options.bloom_bits_per_key = bloom_bits_per_key;
|
|
|
|
plain_table_options.hash_table_ratio = 0.75;
|
|
|
|
options.table_factory = std::shared_ptr<TableFactory>(
|
|
|
|
NewPlainTableFactory(plain_table_options));
|
2014-11-12 22:05:12 +01:00
|
|
|
#else
|
|
|
|
fprintf(stderr, "Plain table is not supported in lite mode\n");
|
|
|
|
exit(1);
|
|
|
|
#endif // ROCKSDB_LITE
|
2014-08-18 20:59:38 +02:00
|
|
|
} else if (FLAGS_use_cuckoo_table) {
|
2014-11-12 22:05:12 +01:00
|
|
|
#ifndef ROCKSDB_LITE
|
2014-08-18 20:59:38 +02:00
|
|
|
if (FLAGS_cuckoo_hash_ratio > 1 || FLAGS_cuckoo_hash_ratio < 0) {
|
|
|
|
fprintf(stderr, "Invalid cuckoo_hash_ratio\n");
|
|
|
|
exit(1);
|
|
|
|
}
|
CuckooTable: add one option to allow identity function for the first hash function
Summary:
MurmurHash becomes expensive when we do millions Get() a second in one
thread. Add this option to allow the first hash function to use identity
function as hash function. It results in QPS increase from 3.7M/s to
~4.3M/s. I did not observe improvement for end to end RocksDB
performance. This may be caused by other bottlenecks that I will address
in a separate diff.
Test Plan:
```
[ljin@dev1964 rocksdb] ./cuckoo_table_reader_test --enable_perf --file_dir=/dev/shm --write --identity_as_first_hash=0
==== Test CuckooReaderTest.WhenKeyExists
==== Test CuckooReaderTest.WhenKeyExistsWithUint64Comparator
==== Test CuckooReaderTest.CheckIterator
==== Test CuckooReaderTest.CheckIteratorUint64
==== Test CuckooReaderTest.WhenKeyNotFound
==== Test CuckooReaderTest.TestReadPerformance
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.272us (3.7 Mqps) with batch size of 0, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.138us (7.2 Mqps) with batch size of 10, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.142us (7.1 Mqps) with batch size of 25, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.142us (7.0 Mqps) with batch size of 50, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.144us (6.9 Mqps) with batch size of 100, # of found keys 125829120
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.201us (5.0 Mqps) with batch size of 0, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.121us (8.3 Mqps) with batch size of 10, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.123us (8.1 Mqps) with batch size of 25, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.121us (8.3 Mqps) with batch size of 50, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.112us (8.9 Mqps) with batch size of 100, # of found keys 104857600
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.251us (4.0 Mqps) with batch size of 0, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.107us (9.4 Mqps) with batch size of 10, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.099us (10.1 Mqps) with batch size of 25, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.100us (10.0 Mqps) with batch size of 50, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.116us (8.6 Mqps) with batch size of 100, # of found keys 83886080
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.189us (5.3 Mqps) with batch size of 0, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.095us (10.5 Mqps) with batch size of 10, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.096us (10.4 Mqps) with batch size of 25, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.098us (10.2 Mqps) with batch size of 50, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.105us (9.5 Mqps) with batch size of 100, # of found keys 73400320
[ljin@dev1964 rocksdb] ./cuckoo_table_reader_test --enable_perf --file_dir=/dev/shm --write --identity_as_first_hash=1
==== Test CuckooReaderTest.WhenKeyExists
==== Test CuckooReaderTest.WhenKeyExistsWithUint64Comparator
==== Test CuckooReaderTest.CheckIterator
==== Test CuckooReaderTest.CheckIteratorUint64
==== Test CuckooReaderTest.WhenKeyNotFound
==== Test CuckooReaderTest.TestReadPerformance
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.230us (4.3 Mqps) with batch size of 0, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.086us (11.7 Mqps) with batch size of 10, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.088us (11.3 Mqps) with batch size of 25, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.083us (12.1 Mqps) with batch size of 50, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.083us (12.1 Mqps) with batch size of 100, # of found keys 125829120
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.159us (6.3 Mqps) with batch size of 0, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 10, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.080us (12.6 Mqps) with batch size of 25, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.080us (12.5 Mqps) with batch size of 50, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.082us (12.2 Mqps) with batch size of 100, # of found keys 104857600
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.154us (6.5 Mqps) with batch size of 0, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.077us (13.0 Mqps) with batch size of 10, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.077us (12.9 Mqps) with batch size of 25, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 50, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.079us (12.6 Mqps) with batch size of 100, # of found keys 83886080
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.218us (4.6 Mqps) with batch size of 0, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.083us (12.0 Mqps) with batch size of 10, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.085us (11.7 Mqps) with batch size of 25, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.086us (11.6 Mqps) with batch size of 50, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 100, # of found keys 73400320
```
Reviewers: sdong, igor, yhchiang
Reviewed By: igor
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D23451
2014-09-18 20:00:48 +02:00
|
|
|
rocksdb::CuckooTableOptions table_options;
|
|
|
|
table_options.hash_table_ratio = FLAGS_cuckoo_hash_ratio;
|
|
|
|
table_options.identity_as_first_hash = FLAGS_identity_as_first_hash;
|
2014-08-18 20:59:38 +02:00
|
|
|
options.table_factory = std::shared_ptr<TableFactory>(
|
CuckooTable: add one option to allow identity function for the first hash function
Summary:
MurmurHash becomes expensive when we do millions Get() a second in one
thread. Add this option to allow the first hash function to use identity
function as hash function. It results in QPS increase from 3.7M/s to
~4.3M/s. I did not observe improvement for end to end RocksDB
performance. This may be caused by other bottlenecks that I will address
in a separate diff.
Test Plan:
```
[ljin@dev1964 rocksdb] ./cuckoo_table_reader_test --enable_perf --file_dir=/dev/shm --write --identity_as_first_hash=0
==== Test CuckooReaderTest.WhenKeyExists
==== Test CuckooReaderTest.WhenKeyExistsWithUint64Comparator
==== Test CuckooReaderTest.CheckIterator
==== Test CuckooReaderTest.CheckIteratorUint64
==== Test CuckooReaderTest.WhenKeyNotFound
==== Test CuckooReaderTest.TestReadPerformance
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.272us (3.7 Mqps) with batch size of 0, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.138us (7.2 Mqps) with batch size of 10, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.142us (7.1 Mqps) with batch size of 25, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.142us (7.0 Mqps) with batch size of 50, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.144us (6.9 Mqps) with batch size of 100, # of found keys 125829120
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.201us (5.0 Mqps) with batch size of 0, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.121us (8.3 Mqps) with batch size of 10, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.123us (8.1 Mqps) with batch size of 25, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.121us (8.3 Mqps) with batch size of 50, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.112us (8.9 Mqps) with batch size of 100, # of found keys 104857600
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.251us (4.0 Mqps) with batch size of 0, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.107us (9.4 Mqps) with batch size of 10, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.099us (10.1 Mqps) with batch size of 25, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.100us (10.0 Mqps) with batch size of 50, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.116us (8.6 Mqps) with batch size of 100, # of found keys 83886080
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.189us (5.3 Mqps) with batch size of 0, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.095us (10.5 Mqps) with batch size of 10, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.096us (10.4 Mqps) with batch size of 25, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.098us (10.2 Mqps) with batch size of 50, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.105us (9.5 Mqps) with batch size of 100, # of found keys 73400320
[ljin@dev1964 rocksdb] ./cuckoo_table_reader_test --enable_perf --file_dir=/dev/shm --write --identity_as_first_hash=1
==== Test CuckooReaderTest.WhenKeyExists
==== Test CuckooReaderTest.WhenKeyExistsWithUint64Comparator
==== Test CuckooReaderTest.CheckIterator
==== Test CuckooReaderTest.CheckIteratorUint64
==== Test CuckooReaderTest.WhenKeyNotFound
==== Test CuckooReaderTest.TestReadPerformance
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.230us (4.3 Mqps) with batch size of 0, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.086us (11.7 Mqps) with batch size of 10, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.088us (11.3 Mqps) with batch size of 25, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.083us (12.1 Mqps) with batch size of 50, # of found keys 125829120
With 125829120 items, utilization is 93.75%, number of hash functions: 2.
Time taken per op is 0.083us (12.1 Mqps) with batch size of 100, # of found keys 125829120
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.159us (6.3 Mqps) with batch size of 0, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 10, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.080us (12.6 Mqps) with batch size of 25, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.080us (12.5 Mqps) with batch size of 50, # of found keys 104857600
With 104857600 items, utilization is 78.12%, number of hash functions: 2.
Time taken per op is 0.082us (12.2 Mqps) with batch size of 100, # of found keys 104857600
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.154us (6.5 Mqps) with batch size of 0, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.077us (13.0 Mqps) with batch size of 10, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.077us (12.9 Mqps) with batch size of 25, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 50, # of found keys 83886080
With 83886080 items, utilization is 62.50%, number of hash functions: 2.
Time taken per op is 0.079us (12.6 Mqps) with batch size of 100, # of found keys 83886080
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.218us (4.6 Mqps) with batch size of 0, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.083us (12.0 Mqps) with batch size of 10, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.085us (11.7 Mqps) with batch size of 25, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.086us (11.6 Mqps) with batch size of 50, # of found keys 73400320
With 73400320 items, utilization is 54.69%, number of hash functions: 2.
Time taken per op is 0.078us (12.8 Mqps) with batch size of 100, # of found keys 73400320
```
Reviewers: sdong, igor, yhchiang
Reviewed By: igor
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D23451
2014-09-18 20:00:48 +02:00
|
|
|
NewCuckooTableFactory(table_options));
|
2014-11-12 22:05:12 +01:00
|
|
|
#else
|
|
|
|
fprintf(stderr, "Cuckoo table is not supported in lite mode\n");
|
|
|
|
exit(1);
|
|
|
|
#endif // ROCKSDB_LITE
|
2014-07-17 02:32:30 +02:00
|
|
|
} else {
|
|
|
|
BlockBasedTableOptions block_based_options;
|
|
|
|
if (FLAGS_use_hash_search) {
|
2014-08-11 19:47:52 +02:00
|
|
|
if (FLAGS_prefix_size == 0) {
|
|
|
|
fprintf(stderr,
|
|
|
|
"prefix_size not assigned when enable use_hash_search \n");
|
|
|
|
exit(1);
|
|
|
|
}
|
2014-07-17 02:32:30 +02:00
|
|
|
block_based_options.index_type = BlockBasedTableOptions::kHashSearch;
|
|
|
|
} else {
|
|
|
|
block_based_options.index_type = BlockBasedTableOptions::kBinarySearch;
|
|
|
|
}
|
2014-08-25 23:22:05 +02:00
|
|
|
if (cache_ == nullptr) {
|
|
|
|
block_based_options.no_block_cache = true;
|
|
|
|
}
|
2015-07-09 22:36:16 +02:00
|
|
|
block_based_options.cache_index_and_filter_blocks =
|
|
|
|
FLAGS_cache_index_and_filter_blocks;
|
2014-08-25 23:22:05 +02:00
|
|
|
block_based_options.block_cache = cache_;
|
|
|
|
block_based_options.block_cache_compressed = compressed_cache_;
|
|
|
|
block_based_options.block_size = FLAGS_block_size;
|
|
|
|
block_based_options.block_restart_interval = FLAGS_block_restart_interval;
|
|
|
|
block_based_options.filter_policy = filter_policy_;
|
2015-11-16 21:56:21 +01:00
|
|
|
block_based_options.skip_table_builder_flush =
|
2015-10-31 02:33:01 +01:00
|
|
|
FLAGS_skip_table_builder_flush;
|
2015-01-15 01:24:24 +01:00
|
|
|
block_based_options.format_version = 2;
|
2014-07-17 02:32:30 +02:00
|
|
|
options.table_factory.reset(
|
|
|
|
NewBlockBasedTableFactory(block_based_options));
|
2014-02-12 19:41:55 +01:00
|
|
|
}
|
2013-10-24 16:43:14 +02:00
|
|
|
if (FLAGS_max_bytes_for_level_multiplier_additional_v.size() > 0) {
|
|
|
|
if (FLAGS_max_bytes_for_level_multiplier_additional_v.size() !=
|
2013-05-21 20:37:06 +02:00
|
|
|
(unsigned int)FLAGS_num_levels) {
|
|
|
|
fprintf(stderr, "Insufficient number of fanouts specified %d\n",
|
2013-10-24 16:43:14 +02:00
|
|
|
(int)FLAGS_max_bytes_for_level_multiplier_additional_v.size());
|
2013-05-21 20:37:06 +02:00
|
|
|
exit(1);
|
|
|
|
}
|
|
|
|
options.max_bytes_for_level_multiplier_additional =
|
2013-10-24 16:43:14 +02:00
|
|
|
FLAGS_max_bytes_for_level_multiplier_additional_v;
|
2013-05-21 20:37:06 +02:00
|
|
|
}
|
2012-08-20 01:05:02 +02:00
|
|
|
options.level0_stop_writes_trigger = FLAGS_level0_stop_writes_trigger;
|
2012-10-23 19:34:09 +02:00
|
|
|
options.level0_file_num_compaction_trigger =
|
2012-10-19 23:00:53 +02:00
|
|
|
FLAGS_level0_file_num_compaction_trigger;
|
2012-08-20 01:05:02 +02:00
|
|
|
options.level0_slowdown_writes_trigger =
|
|
|
|
FLAGS_level0_slowdown_writes_trigger;
|
2013-10-24 16:43:14 +02:00
|
|
|
options.compression = FLAGS_compression_type_e;
|
2013-12-03 21:32:07 +01:00
|
|
|
options.compression_opts.level = FLAGS_compression_level;
|
2013-11-07 03:46:28 +01:00
|
|
|
options.WAL_ttl_seconds = FLAGS_wal_ttl_seconds;
|
|
|
|
options.WAL_size_limit_MB = FLAGS_wal_size_limit_MB;
|
2014-12-23 03:39:28 +01:00
|
|
|
options.max_total_wal_size = FLAGS_max_total_wal_size;
|
|
|
|
|
2012-10-28 07:13:17 +01:00
|
|
|
if (FLAGS_min_level_to_compress >= 0) {
|
|
|
|
assert(FLAGS_min_level_to_compress <= FLAGS_num_levels);
|
2013-01-24 19:54:26 +01:00
|
|
|
options.compression_per_level.resize(FLAGS_num_levels);
|
2012-11-12 22:50:55 +01:00
|
|
|
for (int i = 0; i < FLAGS_min_level_to_compress; i++) {
|
2012-10-28 07:13:17 +01:00
|
|
|
options.compression_per_level[i] = kNoCompression;
|
|
|
|
}
|
2013-03-15 02:32:01 +01:00
|
|
|
for (int i = FLAGS_min_level_to_compress;
|
2012-10-28 07:13:17 +01:00
|
|
|
i < FLAGS_num_levels; i++) {
|
2013-10-24 16:43:14 +02:00
|
|
|
options.compression_per_level[i] = FLAGS_compression_type_e;
|
2012-10-28 07:13:17 +01:00
|
|
|
}
|
|
|
|
}
|
2013-08-06 00:43:49 +02:00
|
|
|
options.soft_rate_limit = FLAGS_soft_rate_limit;
|
|
|
|
options.hard_rate_limit = FLAGS_hard_rate_limit;
|
2015-09-11 23:31:23 +02:00
|
|
|
options.hard_pending_compaction_bytes_limit =
|
|
|
|
FLAGS_hard_pending_compaction_bytes_limit;
|
2015-05-16 00:52:51 +02:00
|
|
|
options.delayed_write_rate = FLAGS_delayed_write_rate;
|
2013-08-06 00:43:49 +02:00
|
|
|
options.rate_limit_delay_max_milliseconds =
|
|
|
|
FLAGS_rate_limit_delay_max_milliseconds;
|
2012-11-01 01:02:24 +01:00
|
|
|
options.table_cache_numshardbits = FLAGS_table_cache_numshardbits;
|
2012-11-29 01:42:36 +01:00
|
|
|
options.max_grandparent_overlap_factor =
|
2012-11-14 18:10:46 +01:00
|
|
|
FLAGS_max_grandparent_overlap_factor;
|
2012-11-21 00:45:41 +01:00
|
|
|
options.disable_auto_compactions = FLAGS_disable_auto_compactions;
|
2012-11-21 08:07:41 +01:00
|
|
|
options.source_compaction_factor = FLAGS_source_compaction_factor;
|
2013-03-15 01:00:04 +01:00
|
|
|
|
|
|
|
// fill storage options
|
2013-10-24 16:43:14 +02:00
|
|
|
options.allow_os_buffer = FLAGS_bufferedio;
|
|
|
|
options.allow_mmap_reads = FLAGS_mmap_read;
|
|
|
|
options.allow_mmap_writes = FLAGS_mmap_write;
|
2013-05-18 00:53:01 +02:00
|
|
|
options.advise_random_on_open = FLAGS_advise_random_on_open;
|
2013-10-24 16:43:14 +02:00
|
|
|
options.access_hint_on_compaction_start = FLAGS_compaction_fadvice_e;
|
2013-06-01 01:30:17 +02:00
|
|
|
options.use_adaptive_mutex = FLAGS_use_adaptive_mutex;
|
2013-06-14 07:49:46 +02:00
|
|
|
options.bytes_per_sync = FLAGS_bytes_per_sync;
|
2015-05-20 01:19:30 +02:00
|
|
|
options.wal_bytes_per_sync = FLAGS_wal_bytes_per_sync;
|
2013-06-01 01:30:17 +02:00
|
|
|
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
// merge operator options
|
2013-08-20 22:35:28 +02:00
|
|
|
options.merge_operator = MergeOperators::CreateFromStringId(
|
|
|
|
FLAGS_merge_operator);
|
|
|
|
if (options.merge_operator == nullptr && !FLAGS_merge_operator.empty()) {
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
fprintf(stderr, "invalid merge operator: %s\n",
|
|
|
|
FLAGS_merge_operator.c_str());
|
|
|
|
exit(1);
|
|
|
|
}
|
2014-01-11 02:33:56 +01:00
|
|
|
options.max_successive_merges = FLAGS_max_successive_merges;
|
Add options.compaction_measure_io_stats to print write I/O stats in compactions
Summary:
Add options.compaction_measure_io_stats to print out / pass to listener accumulated time spent on write calls. Example outputs in info logs:
2015/08/12-16:27:59.463944 7fd428bff700 (Original Log Time 2015/08/12-16:27:59.463922) EVENT_LOG_v1 {"time_micros": 1439422079463897, "job": 6, "event": "compaction_finished", "output_level": 1, "num_output_files": 4, "total_output_size": 6900525, "num_input_records": 111483, "num_output_records": 106877, "file_write_nanos": 15663206, "file_range_sync_nanos": 649588, "file_fsync_nanos": 349614797, "file_prepare_write_nanos": 1505812, "lsm_state": [2, 4, 0, 0, 0, 0, 0]}
Add two more counters in iostats_context.
Also add a parameter of db_bench.
Test Plan: Add a unit test. Also manually verify LOG outputs in db_bench
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D44115
2015-08-13 02:24:45 +02:00
|
|
|
options.compaction_measure_io_stats = FLAGS_compaction_measure_io_stats;
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
|
2013-09-10 01:06:10 +02:00
|
|
|
// set universal style compaction configurations, if applicable
|
|
|
|
if (FLAGS_universal_size_ratio != 0) {
|
|
|
|
options.compaction_options_universal.size_ratio =
|
|
|
|
FLAGS_universal_size_ratio;
|
|
|
|
}
|
|
|
|
if (FLAGS_universal_min_merge_width != 0) {
|
|
|
|
options.compaction_options_universal.min_merge_width =
|
|
|
|
FLAGS_universal_min_merge_width;
|
|
|
|
}
|
|
|
|
if (FLAGS_universal_max_merge_width != 0) {
|
|
|
|
options.compaction_options_universal.max_merge_width =
|
|
|
|
FLAGS_universal_max_merge_width;
|
|
|
|
}
|
|
|
|
if (FLAGS_universal_max_size_amplification_percent != 0) {
|
|
|
|
options.compaction_options_universal.max_size_amplification_percent =
|
|
|
|
FLAGS_universal_max_size_amplification_percent;
|
|
|
|
}
|
2013-12-03 21:32:07 +01:00
|
|
|
if (FLAGS_universal_compression_size_percent != -1) {
|
|
|
|
options.compaction_options_universal.compression_size_percent =
|
|
|
|
FLAGS_universal_compression_size_percent;
|
|
|
|
}
|
2015-07-20 20:52:39 +02:00
|
|
|
options.compaction_options_universal.allow_trivial_move =
|
|
|
|
FLAGS_universal_allow_trivial_move;
|
2015-03-06 20:22:06 +01:00
|
|
|
if (FLAGS_thread_status_per_interval > 0) {
|
|
|
|
options.enable_thread_tracking = true;
|
|
|
|
}
|
2015-05-13 02:40:27 +02:00
|
|
|
if (FLAGS_rate_limiter_bytes_per_sec > 0) {
|
|
|
|
options.rate_limiter.reset(
|
|
|
|
NewGenericRateLimiter(FLAGS_rate_limiter_bytes_per_sec));
|
|
|
|
}
|
2013-09-10 01:06:10 +02:00
|
|
|
|
2015-10-14 21:43:00 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
2015-05-29 23:36:35 +02:00
|
|
|
if (FLAGS_readonly && FLAGS_transaction_db) {
|
|
|
|
fprintf(stderr, "Cannot use readonly flag with transaction_db\n");
|
|
|
|
exit(1);
|
|
|
|
}
|
2015-10-14 21:43:00 +02:00
|
|
|
#endif // ROCKSDB_LITE
|
2015-05-29 23:36:35 +02:00
|
|
|
|
2014-04-11 21:15:09 +02:00
|
|
|
if (FLAGS_num_multi_db <= 1) {
|
|
|
|
OpenDb(options, FLAGS_db, &db_);
|
|
|
|
} else {
|
|
|
|
multi_dbs_.clear();
|
2014-08-19 03:15:01 +02:00
|
|
|
multi_dbs_.resize(FLAGS_num_multi_db);
|
2014-04-29 21:33:57 +02:00
|
|
|
for (int i = 0; i < FLAGS_num_multi_db; i++) {
|
2014-08-19 03:15:01 +02:00
|
|
|
OpenDb(options, GetDbNameForMultiple(FLAGS_db, i), &multi_dbs_[i]);
|
2014-04-11 21:15:09 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
if (FLAGS_min_level_to_compress >= 0) {
|
|
|
|
options.compression_per_level.clear();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-08-19 03:15:01 +02:00
|
|
|
void OpenDb(const Options& options, const std::string& db_name,
|
|
|
|
DBWithColumnFamilies* db) {
|
2012-11-06 04:18:49 +01:00
|
|
|
Status s;
|
2014-08-19 03:15:01 +02:00
|
|
|
// Open with column families if necessary.
|
|
|
|
if (FLAGS_num_column_families > 1) {
|
2014-12-23 03:39:28 +01:00
|
|
|
size_t num_hot = FLAGS_num_column_families;
|
|
|
|
if (FLAGS_num_hot_column_families > 0 &&
|
|
|
|
FLAGS_num_hot_column_families < FLAGS_num_column_families) {
|
|
|
|
num_hot = FLAGS_num_hot_column_families;
|
|
|
|
} else {
|
|
|
|
FLAGS_num_hot_column_families = FLAGS_num_column_families;
|
|
|
|
}
|
2014-08-19 03:15:01 +02:00
|
|
|
std::vector<ColumnFamilyDescriptor> column_families;
|
2014-12-23 03:39:28 +01:00
|
|
|
for (size_t i = 0; i < num_hot; i++) {
|
2014-08-19 03:15:01 +02:00
|
|
|
column_families.push_back(ColumnFamilyDescriptor(
|
|
|
|
ColumnFamilyName(i), ColumnFamilyOptions(options)));
|
|
|
|
}
|
2015-10-14 21:43:00 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
2014-08-19 03:15:01 +02:00
|
|
|
if (FLAGS_readonly) {
|
|
|
|
s = DB::OpenForReadOnly(options, db_name, column_families,
|
|
|
|
&db->cfh, &db->db);
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
} else if (FLAGS_optimistic_transaction_db) {
|
2015-05-29 23:36:35 +02:00
|
|
|
s = OptimisticTransactionDB::Open(options, db_name, column_families,
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
&db->cfh, &db->opt_txn_db);
|
|
|
|
if (s.ok()) {
|
|
|
|
db->db = db->opt_txn_db->GetBaseDB();
|
|
|
|
}
|
|
|
|
} else if (FLAGS_transaction_db) {
|
|
|
|
TransactionDB* ptr;
|
|
|
|
TransactionDBOptions txn_db_options;
|
|
|
|
s = TransactionDB::Open(options, txn_db_options, db_name,
|
|
|
|
column_families, &db->cfh, &ptr);
|
2015-05-29 23:36:35 +02:00
|
|
|
if (s.ok()) {
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
db->db = ptr;
|
2015-05-29 23:36:35 +02:00
|
|
|
}
|
2014-08-19 03:15:01 +02:00
|
|
|
} else {
|
|
|
|
s = DB::Open(options, db_name, column_families, &db->cfh, &db->db);
|
|
|
|
}
|
2015-10-14 21:43:00 +02:00
|
|
|
#else
|
|
|
|
s = DB::Open(options, db_name, column_families, &db->cfh, &db->db);
|
|
|
|
#endif // ROCKSDB_LITE
|
2014-12-23 03:39:28 +01:00
|
|
|
db->cfh.resize(FLAGS_num_column_families);
|
|
|
|
db->num_created = num_hot;
|
|
|
|
db->num_hot = num_hot;
|
2015-10-14 21:43:00 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
2014-08-19 03:15:01 +02:00
|
|
|
} else if (FLAGS_readonly) {
|
|
|
|
s = DB::OpenForReadOnly(options, db_name, &db->db);
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
} else if (FLAGS_optimistic_transaction_db) {
|
|
|
|
s = OptimisticTransactionDB::Open(options, db_name, &db->opt_txn_db);
|
|
|
|
if (s.ok()) {
|
|
|
|
db->db = db->opt_txn_db->GetBaseDB();
|
|
|
|
}
|
2015-05-29 23:36:35 +02:00
|
|
|
} else if (FLAGS_transaction_db) {
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
TransactionDB* ptr;
|
|
|
|
TransactionDBOptions txn_db_options;
|
|
|
|
s = TransactionDB::Open(options, txn_db_options, db_name, &ptr);
|
2015-05-29 23:36:35 +02:00
|
|
|
if (s.ok()) {
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
db->db = ptr;
|
2015-05-29 23:36:35 +02:00
|
|
|
}
|
2015-10-14 21:43:00 +02:00
|
|
|
#endif // ROCKSDB_LITE
|
2012-11-06 04:18:49 +01:00
|
|
|
} else {
|
2014-08-19 03:15:01 +02:00
|
|
|
s = DB::Open(options, db_name, &db->db);
|
2012-11-06 04:18:49 +01:00
|
|
|
}
|
2011-03-22 19:32:49 +01:00
|
|
|
if (!s.ok()) {
|
|
|
|
fprintf(stderr, "open error: %s\n", s.ToString().c_str());
|
|
|
|
exit(1);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2013-06-15 01:17:56 +02:00
|
|
|
enum WriteMode {
|
|
|
|
RANDOM, SEQUENTIAL, UNIQUE_RANDOM
|
|
|
|
};
|
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
void WriteSeq(ThreadState* thread) {
|
2013-06-15 01:17:56 +02:00
|
|
|
DoWrite(thread, SEQUENTIAL);
|
2011-08-22 23:08:51 +02:00
|
|
|
}
|
2011-03-22 19:32:49 +01:00
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
void WriteRandom(ThreadState* thread) {
|
2013-06-15 01:17:56 +02:00
|
|
|
DoWrite(thread, RANDOM);
|
2011-08-22 23:08:51 +02:00
|
|
|
}
|
|
|
|
|
2013-06-15 01:17:56 +02:00
|
|
|
void WriteUniqueRandom(ThreadState* thread) {
|
|
|
|
DoWrite(thread, UNIQUE_RANDOM);
|
|
|
|
}
|
|
|
|
|
2014-04-09 20:25:21 +02:00
|
|
|
class KeyGenerator {
|
|
|
|
public:
|
|
|
|
KeyGenerator(Random64* rand, WriteMode mode,
|
|
|
|
uint64_t num, uint64_t num_per_set = 64 * 1024)
|
|
|
|
: rand_(rand),
|
|
|
|
mode_(mode),
|
|
|
|
num_(num),
|
|
|
|
next_(0) {
|
|
|
|
if (mode_ == UNIQUE_RANDOM) {
|
|
|
|
// NOTE: if memory consumption of this approach becomes a concern,
|
|
|
|
// we can either break it into pieces and only random shuffle a section
|
|
|
|
// each time. Alternatively, use a bit map implementation
|
|
|
|
// (https://reviews.facebook.net/differential/diff/54627/)
|
|
|
|
values_.resize(num_);
|
|
|
|
for (uint64_t i = 0; i < num_; ++i) {
|
|
|
|
values_[i] = i;
|
|
|
|
}
|
2014-11-11 22:47:22 +01:00
|
|
|
std::shuffle(
|
|
|
|
values_.begin(), values_.end(),
|
|
|
|
std::default_random_engine(static_cast<unsigned int>(FLAGS_seed)));
|
2014-04-09 20:25:21 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
uint64_t Next() {
|
|
|
|
switch (mode_) {
|
|
|
|
case SEQUENTIAL:
|
|
|
|
return next_++;
|
|
|
|
case RANDOM:
|
|
|
|
return rand_->Next() % num_;
|
|
|
|
case UNIQUE_RANDOM:
|
|
|
|
return values_[next_++];
|
|
|
|
}
|
|
|
|
assert(false);
|
|
|
|
return std::numeric_limits<uint64_t>::max();
|
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
Random64* rand_;
|
|
|
|
WriteMode mode_;
|
|
|
|
const uint64_t num_;
|
|
|
|
uint64_t next_;
|
|
|
|
std::vector<uint64_t> values_;
|
|
|
|
};
|
|
|
|
|
2014-04-11 21:15:09 +02:00
|
|
|
DB* SelectDB(ThreadState* thread) {
|
2014-08-19 03:15:01 +02:00
|
|
|
return SelectDBWithCfh(thread)->db;
|
|
|
|
}
|
|
|
|
|
|
|
|
DBWithColumnFamilies* SelectDBWithCfh(ThreadState* thread) {
|
|
|
|
return SelectDBWithCfh(thread->rand.Next());
|
|
|
|
}
|
|
|
|
|
|
|
|
DBWithColumnFamilies* SelectDBWithCfh(uint64_t rand_int) {
|
|
|
|
if (db_.db != nullptr) {
|
|
|
|
return &db_;
|
|
|
|
} else {
|
|
|
|
return &multi_dbs_[rand_int % multi_dbs_.size()];
|
2014-04-11 21:15:09 +02:00
|
|
|
}
|
|
|
|
}
|
2014-04-09 20:25:21 +02:00
|
|
|
|
2013-06-15 01:17:56 +02:00
|
|
|
void DoWrite(ThreadState* thread, WriteMode write_mode) {
|
|
|
|
const int test_duration = write_mode == RANDOM ? FLAGS_duration : 0;
|
2014-03-05 19:27:17 +01:00
|
|
|
const int64_t num_ops = writes_ == 0 ? num_ : writes_;
|
2013-06-15 01:17:56 +02:00
|
|
|
|
2014-04-11 21:15:09 +02:00
|
|
|
size_t num_key_gens = 1;
|
2014-08-19 03:15:01 +02:00
|
|
|
if (db_.db == nullptr) {
|
2014-04-11 21:15:09 +02:00
|
|
|
num_key_gens = multi_dbs_.size();
|
|
|
|
}
|
|
|
|
std::vector<std::unique_ptr<KeyGenerator>> key_gens(num_key_gens);
|
2014-12-23 03:39:28 +01:00
|
|
|
int64_t max_ops = num_ops * num_key_gens;
|
|
|
|
int64_t ops_per_stage = max_ops;
|
|
|
|
if (FLAGS_num_column_families > 1 && FLAGS_num_hot_column_families > 0) {
|
|
|
|
ops_per_stage = (max_ops - 1) / (FLAGS_num_column_families /
|
|
|
|
FLAGS_num_hot_column_families) +
|
|
|
|
1;
|
|
|
|
}
|
|
|
|
|
|
|
|
Duration duration(test_duration, max_ops, ops_per_stage);
|
2014-04-11 21:15:09 +02:00
|
|
|
for (size_t i = 0; i < num_key_gens; i++) {
|
2015-02-26 00:53:45 +01:00
|
|
|
key_gens[i].reset(new KeyGenerator(&(thread->rand), write_mode, num_,
|
2014-12-23 03:39:28 +01:00
|
|
|
ops_per_stage));
|
2014-04-11 21:15:09 +02:00
|
|
|
}
|
2013-03-01 20:25:13 +01:00
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
if (num_ != FLAGS_num) {
|
2011-03-22 19:32:49 +01:00
|
|
|
char msg[100];
|
2014-03-05 02:08:05 +01:00
|
|
|
snprintf(msg, sizeof(msg), "(%" PRIu64 " ops)", num_);
|
2011-08-22 23:08:51 +02:00
|
|
|
thread->stats.AddMessage(msg);
|
2011-03-22 19:32:49 +01:00
|
|
|
}
|
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
RandomGenerator gen;
|
2011-03-18 23:37:00 +01:00
|
|
|
WriteBatch batch;
|
|
|
|
Status s;
|
2011-08-22 23:08:51 +02:00
|
|
|
int64_t bytes = 0;
|
2014-04-08 20:21:09 +02:00
|
|
|
|
2015-02-19 23:27:48 +01:00
|
|
|
std::unique_ptr<const char[]> key_guard;
|
|
|
|
Slice key = AllocateKey(&key_guard);
|
2014-12-23 03:39:28 +01:00
|
|
|
int64_t stage = 0;
|
2013-03-01 20:25:13 +01:00
|
|
|
while (!duration.Done(entries_per_batch_)) {
|
2014-12-23 03:39:28 +01:00
|
|
|
if (duration.GetStage() != stage) {
|
|
|
|
stage = duration.GetStage();
|
|
|
|
if (db_.db != nullptr) {
|
|
|
|
db_.CreateNewCf(open_options_, stage);
|
|
|
|
} else {
|
|
|
|
for (auto& db : multi_dbs_) {
|
|
|
|
db.CreateNewCf(open_options_, stage);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2014-08-19 03:15:01 +02:00
|
|
|
size_t id = thread->rand.Next() % num_key_gens;
|
|
|
|
DBWithColumnFamilies* db_with_cfh = SelectDBWithCfh(id);
|
2011-03-18 23:37:00 +01:00
|
|
|
batch.Clear();
|
2014-04-08 20:21:09 +02:00
|
|
|
for (int64_t j = 0; j < entries_per_batch_; j++) {
|
2015-06-18 01:44:52 +02:00
|
|
|
if (thread->shared->write_rate_limiter.get() != nullptr) {
|
|
|
|
thread->shared->write_rate_limiter->Request(value_size_ + key_size_,
|
|
|
|
Env::IO_HIGH);
|
|
|
|
}
|
2014-08-19 03:15:01 +02:00
|
|
|
int64_t rand_num = key_gens[id]->Next();
|
|
|
|
GenerateKeyFromInt(rand_num, FLAGS_num, &key);
|
|
|
|
if (FLAGS_num_column_families <= 1) {
|
|
|
|
batch.Put(key, gen.Generate(value_size_));
|
|
|
|
} else {
|
|
|
|
// We use same rand_num as seed for key and column family so that we
|
|
|
|
// can deterministically find the cfh corresponding to a particular
|
|
|
|
// key while reading the key.
|
2014-12-23 03:39:28 +01:00
|
|
|
batch.Put(db_with_cfh->GetCfh(rand_num), key,
|
|
|
|
gen.Generate(value_size_));
|
2014-08-19 03:15:01 +02:00
|
|
|
}
|
2014-04-08 20:21:09 +02:00
|
|
|
bytes += value_size_ + key_size_;
|
2011-04-12 21:38:58 +02:00
|
|
|
}
|
2014-08-19 03:15:01 +02:00
|
|
|
s = db_with_cfh->db->Write(write_options_, &batch);
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(db_with_cfh, db_with_cfh->db,
|
|
|
|
entries_per_batch_);
|
2011-03-18 23:37:00 +01:00
|
|
|
if (!s.ok()) {
|
|
|
|
fprintf(stderr, "put error: %s\n", s.ToString().c_str());
|
|
|
|
exit(1);
|
|
|
|
}
|
|
|
|
}
|
2011-08-22 23:08:51 +02:00
|
|
|
thread->stats.AddBytes(bytes);
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
void ReadSequential(ThreadState* thread) {
|
2014-08-19 03:15:01 +02:00
|
|
|
if (db_.db != nullptr) {
|
|
|
|
ReadSequential(thread, db_.db);
|
2014-04-11 21:15:09 +02:00
|
|
|
} else {
|
2014-08-19 03:15:01 +02:00
|
|
|
for (const auto& db_with_cfh : multi_dbs_) {
|
|
|
|
ReadSequential(thread, db_with_cfh.db);
|
2014-04-11 21:15:09 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void ReadSequential(ThreadState* thread, DB* db) {
|
2014-09-26 23:20:24 +02:00
|
|
|
ReadOptions options(FLAGS_verify_checksum, true);
|
|
|
|
options.tailing = FLAGS_use_tailing_iterator;
|
|
|
|
|
|
|
|
Iterator* iter = db->NewIterator(options);
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t i = 0;
|
2011-08-22 23:08:51 +02:00
|
|
|
int64_t bytes = 0;
|
2011-05-21 04:17:43 +02:00
|
|
|
for (iter->SeekToFirst(); i < reads_ && iter->Valid(); iter->Next()) {
|
2011-08-22 23:08:51 +02:00
|
|
|
bytes += iter->key().size() + iter->value().size();
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(nullptr, db, 1);
|
2011-03-22 19:32:49 +01:00
|
|
|
++i;
|
|
|
|
}
|
|
|
|
delete iter;
|
2011-08-22 23:08:51 +02:00
|
|
|
thread->stats.AddBytes(bytes);
|
2011-03-22 19:32:49 +01:00
|
|
|
}
|
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
void ReadReverse(ThreadState* thread) {
|
2014-08-19 03:15:01 +02:00
|
|
|
if (db_.db != nullptr) {
|
|
|
|
ReadReverse(thread, db_.db);
|
2014-04-11 21:15:09 +02:00
|
|
|
} else {
|
2014-08-19 03:15:01 +02:00
|
|
|
for (const auto& db_with_cfh : multi_dbs_) {
|
|
|
|
ReadReverse(thread, db_with_cfh.db);
|
2014-04-11 21:15:09 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void ReadReverse(ThreadState* thread, DB* db) {
|
|
|
|
Iterator* iter = db->NewIterator(ReadOptions(FLAGS_verify_checksum, true));
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t i = 0;
|
2011-08-22 23:08:51 +02:00
|
|
|
int64_t bytes = 0;
|
2011-05-21 04:17:43 +02:00
|
|
|
for (iter->SeekToLast(); i < reads_ && iter->Valid(); iter->Prev()) {
|
2011-08-22 23:08:51 +02:00
|
|
|
bytes += iter->key().size() + iter->value().size();
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(nullptr, db, 1);
|
2011-03-22 19:32:49 +01:00
|
|
|
++i;
|
|
|
|
}
|
|
|
|
delete iter;
|
2011-08-22 23:08:51 +02:00
|
|
|
thread->stats.AddBytes(bytes);
|
2011-03-22 19:32:49 +01:00
|
|
|
}
|
|
|
|
|
2014-09-25 20:14:01 +02:00
|
|
|
void ReadRandomFast(ThreadState* thread) {
|
|
|
|
int64_t read = 0;
|
|
|
|
int64_t found = 0;
|
2014-09-25 22:34:51 +02:00
|
|
|
int64_t nonexist = 0;
|
2014-09-25 20:14:01 +02:00
|
|
|
ReadOptions options(FLAGS_verify_checksum, true);
|
2015-02-19 23:27:48 +01:00
|
|
|
std::unique_ptr<const char[]> key_guard;
|
|
|
|
Slice key = AllocateKey(&key_guard);
|
2014-09-25 20:14:01 +02:00
|
|
|
std::string value;
|
|
|
|
DB* db = SelectDBWithCfh(thread)->db;
|
|
|
|
|
|
|
|
int64_t pot = 1;
|
|
|
|
while (pot < FLAGS_num) {
|
|
|
|
pot <<= 1;
|
|
|
|
}
|
|
|
|
|
|
|
|
Duration duration(FLAGS_duration, reads_);
|
|
|
|
do {
|
|
|
|
for (int i = 0; i < 100; ++i) {
|
|
|
|
int64_t key_rand = thread->rand.Next() & (pot - 1);
|
|
|
|
GenerateKeyFromInt(key_rand, FLAGS_num, &key);
|
|
|
|
++read;
|
2015-01-22 03:18:15 +01:00
|
|
|
auto status = db->Get(options, key, &value);
|
|
|
|
if (status.ok()) {
|
2014-09-25 20:14:01 +02:00
|
|
|
++found;
|
2015-01-22 03:18:15 +01:00
|
|
|
} else if (!status.IsNotFound()) {
|
2015-01-22 03:23:12 +01:00
|
|
|
fprintf(stderr, "Get returned an error: %s\n",
|
|
|
|
status.ToString().c_str());
|
2015-01-22 03:18:15 +01:00
|
|
|
abort();
|
2014-09-25 20:14:01 +02:00
|
|
|
}
|
2014-09-25 22:34:51 +02:00
|
|
|
if (key_rand >= FLAGS_num) {
|
|
|
|
++nonexist;
|
|
|
|
}
|
2014-09-25 20:14:01 +02:00
|
|
|
}
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(nullptr, db, 100);
|
2014-09-25 20:14:01 +02:00
|
|
|
} while (!duration.Done(100));
|
|
|
|
|
|
|
|
char msg[100];
|
2014-09-25 22:34:51 +02:00
|
|
|
snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found, "
|
|
|
|
"issued %" PRIu64 " non-exist keys)\n",
|
|
|
|
found, read, nonexist);
|
2014-09-25 20:14:01 +02:00
|
|
|
|
|
|
|
thread->stats.AddMessage(msg);
|
|
|
|
|
|
|
|
if (FLAGS_perf_level > 0) {
|
|
|
|
thread->stats.AddMessage(perf_context.ToString());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-03-07 01:38:50 +01:00
|
|
|
int64_t GetRandomKey(Random64* rand) {
|
|
|
|
uint64_t rand_int = rand->Next();
|
|
|
|
int64_t key_rand;
|
|
|
|
if (read_random_exp_range_ == 0) {
|
|
|
|
key_rand = rand_int % FLAGS_num;
|
|
|
|
} else {
|
|
|
|
const uint64_t kBigInt = static_cast<uint64_t>(1U) << 62;
|
|
|
|
long double order = -static_cast<long double>(rand_int % kBigInt) /
|
|
|
|
static_cast<long double>(kBigInt) *
|
|
|
|
read_random_exp_range_;
|
|
|
|
long double exp_ran = std::exp(order);
|
2015-03-11 00:32:12 +01:00
|
|
|
uint64_t rand_num =
|
2015-03-07 01:38:50 +01:00
|
|
|
static_cast<int64_t>(exp_ran * static_cast<long double>(FLAGS_num));
|
2015-03-11 00:32:12 +01:00
|
|
|
// Map to a different number to avoid locality.
|
|
|
|
const uint64_t kBigPrime = 0x5bd1e995;
|
|
|
|
// Overflow is like %(2^64). Will have little impact of results.
|
|
|
|
key_rand = static_cast<int64_t>((rand_num * kBigPrime) % FLAGS_num);
|
2015-03-07 01:38:50 +01:00
|
|
|
}
|
|
|
|
return key_rand;
|
|
|
|
}
|
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
void ReadRandom(ThreadState* thread) {
|
2014-03-28 17:21:20 +01:00
|
|
|
int64_t read = 0;
|
2014-04-08 20:21:09 +02:00
|
|
|
int64_t found = 0;
|
2015-03-30 20:28:25 +02:00
|
|
|
int64_t bytes = 0;
|
2014-04-08 20:21:09 +02:00
|
|
|
ReadOptions options(FLAGS_verify_checksum, true);
|
2015-02-19 23:27:48 +01:00
|
|
|
std::unique_ptr<const char[]> key_guard;
|
|
|
|
Slice key = AllocateKey(&key_guard);
|
2014-04-08 20:21:09 +02:00
|
|
|
std::string value;
|
2013-01-03 21:11:50 +01:00
|
|
|
|
2014-04-08 20:21:09 +02:00
|
|
|
Duration duration(FLAGS_duration, reads_);
|
|
|
|
while (!duration.Done(1)) {
|
2014-08-19 03:15:01 +02:00
|
|
|
DBWithColumnFamilies* db_with_cfh = SelectDBWithCfh(thread);
|
|
|
|
// We use same key_rand as seed for key and column family so that we can
|
|
|
|
// deterministically find the cfh corresponding to a particular key, as it
|
|
|
|
// is done in DoWrite method.
|
2015-03-07 01:38:50 +01:00
|
|
|
int64_t key_rand = GetRandomKey(&thread->rand);
|
2014-08-19 03:15:01 +02:00
|
|
|
GenerateKeyFromInt(key_rand, FLAGS_num, &key);
|
2014-04-08 20:21:09 +02:00
|
|
|
read++;
|
2014-08-19 03:15:01 +02:00
|
|
|
Status s;
|
|
|
|
if (FLAGS_num_column_families > 1) {
|
2014-12-23 03:39:28 +01:00
|
|
|
s = db_with_cfh->db->Get(options, db_with_cfh->GetCfh(key_rand), key,
|
|
|
|
&value);
|
2014-08-19 03:15:01 +02:00
|
|
|
} else {
|
|
|
|
s = db_with_cfh->db->Get(options, key, &value);
|
|
|
|
}
|
|
|
|
if (s.ok()) {
|
2014-04-08 20:21:09 +02:00
|
|
|
found++;
|
2015-03-30 20:28:25 +02:00
|
|
|
bytes += key.size() + value.size();
|
2015-01-22 03:18:15 +01:00
|
|
|
} else if (!s.IsNotFound()) {
|
2015-01-22 03:23:12 +01:00
|
|
|
fprintf(stderr, "Get returned an error: %s\n", s.ToString().c_str());
|
2015-01-22 03:18:15 +01:00
|
|
|
abort();
|
2013-03-01 20:25:13 +01:00
|
|
|
}
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(db_with_cfh, db_with_cfh->db, 1);
|
2012-04-17 17:36:46 +02:00
|
|
|
}
|
2013-06-12 21:42:21 +02:00
|
|
|
|
2012-04-17 17:36:46 +02:00
|
|
|
char msg[100];
|
2014-05-30 23:31:55 +02:00
|
|
|
snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found)\n",
|
2014-03-28 17:21:20 +01:00
|
|
|
found, read);
|
2014-03-05 19:32:54 +01:00
|
|
|
|
2015-03-30 20:28:25 +02:00
|
|
|
thread->stats.AddBytes(bytes);
|
2012-04-17 17:36:46 +02:00
|
|
|
thread->stats.AddMessage(msg);
|
2014-03-05 19:32:54 +01:00
|
|
|
|
|
|
|
if (FLAGS_perf_level > 0) {
|
|
|
|
thread->stats.AddMessage(perf_context.ToString());
|
|
|
|
}
|
2012-04-17 17:36:46 +02:00
|
|
|
}
|
|
|
|
|
2014-04-08 20:21:09 +02:00
|
|
|
// Calls MultiGet over a list of keys from a random distribution.
|
|
|
|
// Returns the total number of keys found.
|
|
|
|
void MultiReadRandom(ThreadState* thread) {
|
|
|
|
int64_t read = 0;
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t found = 0;
|
2012-05-19 09:05:48 +02:00
|
|
|
ReadOptions options(FLAGS_verify_checksum, true);
|
2014-04-15 00:42:20 +02:00
|
|
|
std::vector<Slice> keys;
|
2015-02-19 23:27:48 +01:00
|
|
|
std::vector<std::unique_ptr<const char[]> > key_guards;
|
2014-04-08 20:21:09 +02:00
|
|
|
std::vector<std::string> values(entries_per_batch_);
|
2014-04-29 21:33:57 +02:00
|
|
|
while (static_cast<int64_t>(keys.size()) < entries_per_batch_) {
|
2015-02-19 23:27:48 +01:00
|
|
|
key_guards.push_back(std::move(std::unique_ptr<const char[]>()));
|
|
|
|
keys.push_back(AllocateKey(&key_guards.back()));
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2013-03-01 20:25:13 +01:00
|
|
|
Duration duration(FLAGS_duration, reads_);
|
2014-04-08 20:21:09 +02:00
|
|
|
while (!duration.Done(1)) {
|
2014-04-11 21:15:09 +02:00
|
|
|
DB* db = SelectDB(thread);
|
2014-04-08 20:21:09 +02:00
|
|
|
for (int64_t i = 0; i < entries_per_batch_; ++i) {
|
2015-03-07 01:38:50 +01:00
|
|
|
GenerateKeyFromInt(GetRandomKey(&thread->rand), FLAGS_num, &keys[i]);
|
2013-06-12 21:42:21 +02:00
|
|
|
}
|
2014-04-11 21:15:09 +02:00
|
|
|
std::vector<Status> statuses = db->MultiGet(options, keys, &values);
|
2014-04-29 21:33:57 +02:00
|
|
|
assert(static_cast<int64_t>(statuses.size()) == entries_per_batch_);
|
2014-04-08 20:21:09 +02:00
|
|
|
|
|
|
|
read += entries_per_batch_;
|
|
|
|
for (int64_t i = 0; i < entries_per_batch_; ++i) {
|
|
|
|
if (statuses[i].ok()) {
|
2013-06-12 21:42:21 +02:00
|
|
|
++found;
|
2015-01-22 03:18:15 +01:00
|
|
|
} else if (!statuses[i].IsNotFound()) {
|
|
|
|
fprintf(stderr, "MultiGet returned an error: %s\n",
|
|
|
|
statuses[i].ToString().c_str());
|
|
|
|
abort();
|
2013-06-12 21:42:21 +02:00
|
|
|
}
|
|
|
|
}
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(nullptr, db, entries_per_batch_);
|
2011-05-21 04:17:43 +02:00
|
|
|
}
|
2013-06-12 21:42:21 +02:00
|
|
|
|
|
|
|
char msg[100];
|
2014-03-05 02:08:05 +01:00
|
|
|
snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found)",
|
2014-04-08 20:21:09 +02:00
|
|
|
found, read);
|
2013-06-12 21:42:21 +02:00
|
|
|
thread->stats.AddMessage(msg);
|
2011-05-21 04:17:43 +02:00
|
|
|
}
|
|
|
|
|
2014-02-03 22:13:36 +01:00
|
|
|
void IteratorCreation(ThreadState* thread) {
|
|
|
|
Duration duration(FLAGS_duration, reads_);
|
|
|
|
ReadOptions options(FLAGS_verify_checksum, true);
|
|
|
|
while (!duration.Done(1)) {
|
2014-04-11 21:15:09 +02:00
|
|
|
DB* db = SelectDB(thread);
|
|
|
|
Iterator* iter = db->NewIterator(options);
|
2014-02-03 22:13:36 +01:00
|
|
|
delete iter;
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(nullptr, db, 1);
|
2014-02-03 22:13:36 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-04-10 19:15:59 +02:00
|
|
|
void IteratorCreationWhileWriting(ThreadState* thread) {
|
|
|
|
if (thread->tid > 0) {
|
|
|
|
IteratorCreation(thread);
|
|
|
|
} else {
|
2015-03-18 21:50:52 +01:00
|
|
|
BGWriter(thread, kPut);
|
2014-04-10 19:15:59 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2012-04-17 17:36:46 +02:00
|
|
|
void SeekRandom(ThreadState* thread) {
|
2014-04-08 20:21:09 +02:00
|
|
|
int64_t read = 0;
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t found = 0;
|
2015-03-30 20:28:25 +02:00
|
|
|
int64_t bytes = 0;
|
2014-04-08 20:21:09 +02:00
|
|
|
ReadOptions options(FLAGS_verify_checksum, true);
|
|
|
|
options.tailing = FLAGS_use_tailing_iterator;
|
2014-04-11 21:15:09 +02:00
|
|
|
|
|
|
|
Iterator* single_iter = nullptr;
|
|
|
|
std::vector<Iterator*> multi_iters;
|
2014-08-19 03:15:01 +02:00
|
|
|
if (db_.db != nullptr) {
|
|
|
|
single_iter = db_.db->NewIterator(options);
|
2014-04-11 21:15:09 +02:00
|
|
|
} else {
|
2014-08-19 03:15:01 +02:00
|
|
|
for (const auto& db_with_cfh : multi_dbs_) {
|
|
|
|
multi_iters.push_back(db_with_cfh.db->NewIterator(options));
|
2014-04-11 21:15:09 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-02-19 23:27:48 +01:00
|
|
|
std::unique_ptr<const char[]> key_guard;
|
|
|
|
Slice key = AllocateKey(&key_guard);
|
2014-04-08 20:21:09 +02:00
|
|
|
|
|
|
|
Duration duration(FLAGS_duration, reads_);
|
2014-10-17 19:20:25 +02:00
|
|
|
char value_buffer[256];
|
2013-03-01 20:25:13 +01:00
|
|
|
while (!duration.Done(1)) {
|
2015-05-01 23:17:45 +02:00
|
|
|
if (!FLAGS_use_tailing_iterator) {
|
|
|
|
if (db_.db != nullptr) {
|
|
|
|
delete single_iter;
|
|
|
|
single_iter = db_.db->NewIterator(options);
|
|
|
|
} else {
|
|
|
|
for (auto iter : multi_iters) {
|
|
|
|
delete iter;
|
|
|
|
}
|
|
|
|
multi_iters.clear();
|
|
|
|
for (const auto& db_with_cfh : multi_dbs_) {
|
|
|
|
multi_iters.push_back(db_with_cfh.db->NewIterator(options));
|
2014-05-30 23:09:22 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2014-04-11 21:15:09 +02:00
|
|
|
// Pick a Iterator to use
|
|
|
|
Iterator* iter_to_use = single_iter;
|
|
|
|
if (single_iter == nullptr) {
|
|
|
|
iter_to_use = multi_iters[thread->rand.Next() % multi_iters.size()];
|
|
|
|
}
|
|
|
|
|
2014-04-08 20:21:09 +02:00
|
|
|
GenerateKeyFromInt(thread->rand.Next() % FLAGS_num, FLAGS_num, &key);
|
2014-04-11 21:15:09 +02:00
|
|
|
iter_to_use->Seek(key);
|
2014-04-08 20:21:09 +02:00
|
|
|
read++;
|
2014-04-11 21:15:09 +02:00
|
|
|
if (iter_to_use->Valid() && iter_to_use->key().compare(key) == 0) {
|
2014-04-08 20:21:09 +02:00
|
|
|
found++;
|
|
|
|
}
|
2014-10-17 19:20:25 +02:00
|
|
|
|
|
|
|
for (int j = 0; j < FLAGS_seek_nexts && iter_to_use->Valid(); ++j) {
|
|
|
|
// Copy out iterator's value to make sure we read them.
|
|
|
|
Slice value = iter_to_use->value();
|
|
|
|
memcpy(value_buffer, value.data(),
|
|
|
|
std::min(value.size(), sizeof(value_buffer)));
|
2015-03-30 20:28:25 +02:00
|
|
|
bytes += iter_to_use->key().size() + iter_to_use->value().size();
|
2015-03-18 21:50:52 +01:00
|
|
|
|
|
|
|
if (!FLAGS_reverse_iterator) {
|
|
|
|
iter_to_use->Next();
|
|
|
|
} else {
|
|
|
|
iter_to_use->Prev();
|
|
|
|
}
|
2014-10-17 19:20:25 +02:00
|
|
|
assert(iter_to_use->status().ok());
|
|
|
|
}
|
|
|
|
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(&db_, db_.db, 1);
|
2012-04-17 17:36:46 +02:00
|
|
|
}
|
2014-04-11 21:15:09 +02:00
|
|
|
delete single_iter;
|
|
|
|
for (auto iter : multi_iters) {
|
|
|
|
delete iter;
|
|
|
|
}
|
2014-04-08 20:21:09 +02:00
|
|
|
|
2012-04-17 17:36:46 +02:00
|
|
|
char msg[100];
|
2014-05-30 23:31:55 +02:00
|
|
|
snprintf(msg, sizeof(msg), "(%" PRIu64 " of %" PRIu64 " found)\n",
|
2014-04-08 20:21:09 +02:00
|
|
|
found, read);
|
2015-03-30 20:28:25 +02:00
|
|
|
thread->stats.AddBytes(bytes);
|
2012-04-17 17:36:46 +02:00
|
|
|
thread->stats.AddMessage(msg);
|
2014-05-30 23:31:55 +02:00
|
|
|
if (FLAGS_perf_level > 0) {
|
|
|
|
thread->stats.AddMessage(perf_context.ToString());
|
|
|
|
}
|
2012-04-17 17:36:46 +02:00
|
|
|
}
|
2014-04-11 18:47:20 +02:00
|
|
|
|
|
|
|
void SeekRandomWhileWriting(ThreadState* thread) {
|
|
|
|
if (thread->tid > 0) {
|
|
|
|
SeekRandom(thread);
|
|
|
|
} else {
|
2015-03-18 21:50:52 +01:00
|
|
|
BGWriter(thread, kPut);
|
2014-04-11 18:47:20 +02:00
|
|
|
}
|
|
|
|
}
|
2012-04-17 17:36:46 +02:00
|
|
|
|
2015-03-30 20:28:25 +02:00
|
|
|
void SeekRandomWhileMerging(ThreadState* thread) {
|
|
|
|
if (thread->tid > 0) {
|
|
|
|
SeekRandom(thread);
|
|
|
|
} else {
|
|
|
|
BGWriter(thread, kMerge);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2012-04-17 17:36:46 +02:00
|
|
|
void DoDelete(ThreadState* thread, bool seq) {
|
|
|
|
WriteBatch batch;
|
2013-03-01 20:25:13 +01:00
|
|
|
Duration duration(seq ? 0 : FLAGS_duration, num_);
|
2014-04-08 20:21:09 +02:00
|
|
|
int64_t i = 0;
|
2015-02-19 23:27:48 +01:00
|
|
|
std::unique_ptr<const char[]> key_guard;
|
|
|
|
Slice key = AllocateKey(&key_guard);
|
2014-04-08 20:21:09 +02:00
|
|
|
|
2013-03-01 20:25:13 +01:00
|
|
|
while (!duration.Done(entries_per_batch_)) {
|
2014-04-11 21:15:09 +02:00
|
|
|
DB* db = SelectDB(thread);
|
2012-04-17 17:36:46 +02:00
|
|
|
batch.Clear();
|
2014-04-08 20:21:09 +02:00
|
|
|
for (int64_t j = 0; j < entries_per_batch_; ++j) {
|
|
|
|
const int64_t k = seq ? i + j : (thread->rand.Next() % FLAGS_num);
|
|
|
|
GenerateKeyFromInt(k, FLAGS_num, &key);
|
2014-03-05 02:08:05 +01:00
|
|
|
batch.Delete(key);
|
2012-04-17 17:36:46 +02:00
|
|
|
}
|
2014-04-11 21:15:09 +02:00
|
|
|
auto s = db->Write(write_options_, &batch);
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(nullptr, db, entries_per_batch_);
|
2012-04-17 17:36:46 +02:00
|
|
|
if (!s.ok()) {
|
|
|
|
fprintf(stderr, "del error: %s\n", s.ToString().c_str());
|
|
|
|
exit(1);
|
|
|
|
}
|
2014-04-08 20:21:09 +02:00
|
|
|
i += entries_per_batch_;
|
2012-04-17 17:36:46 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void DeleteSeq(ThreadState* thread) {
|
|
|
|
DoDelete(thread, true);
|
|
|
|
}
|
|
|
|
|
|
|
|
void DeleteRandom(ThreadState* thread) {
|
|
|
|
DoDelete(thread, false);
|
|
|
|
}
|
|
|
|
|
2011-09-01 21:08:02 +02:00
|
|
|
void ReadWhileWriting(ThreadState* thread) {
|
|
|
|
if (thread->tid > 0) {
|
|
|
|
ReadRandom(thread);
|
|
|
|
} else {
|
2015-03-18 21:50:52 +01:00
|
|
|
BGWriter(thread, kPut);
|
2014-04-10 19:15:59 +02:00
|
|
|
}
|
|
|
|
}
|
2011-09-01 21:08:02 +02:00
|
|
|
|
2015-03-18 21:50:52 +01:00
|
|
|
void ReadWhileMerging(ThreadState* thread) {
|
|
|
|
if (thread->tid > 0) {
|
|
|
|
ReadRandom(thread);
|
|
|
|
} else {
|
|
|
|
BGWriter(thread, kMerge);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void BGWriter(ThreadState* thread, enum PutOrMerge write_merge) {
|
2014-04-10 19:15:59 +02:00
|
|
|
// Special thread that keeps writing until other threads are done.
|
|
|
|
RandomGenerator gen;
|
|
|
|
double last = FLAGS_env->NowMicros();
|
|
|
|
int writes_per_second_by_10 = 0;
|
|
|
|
int num_writes = 0;
|
2015-03-30 20:28:25 +02:00
|
|
|
int64_t bytes = 0;
|
2014-04-10 19:15:59 +02:00
|
|
|
|
|
|
|
// --writes_per_second rate limit is enforced per 100 milliseconds
|
|
|
|
// intervals to avoid a burst of writes at the start of each second.
|
|
|
|
|
|
|
|
if (FLAGS_writes_per_second > 0)
|
|
|
|
writes_per_second_by_10 = FLAGS_writes_per_second / 10;
|
|
|
|
|
|
|
|
// Don't merge stats from this thread with the readers.
|
|
|
|
thread->stats.SetExcludeFromMerge();
|
|
|
|
|
2015-02-19 23:27:48 +01:00
|
|
|
std::unique_ptr<const char[]> key_guard;
|
|
|
|
Slice key = AllocateKey(&key_guard);
|
2014-04-10 19:15:59 +02:00
|
|
|
|
|
|
|
while (true) {
|
2014-04-11 21:15:09 +02:00
|
|
|
DB* db = SelectDB(thread);
|
2014-04-10 19:15:59 +02:00
|
|
|
{
|
|
|
|
MutexLock l(&thread->shared->mu);
|
|
|
|
if (thread->shared->num_done + 1 >= thread->shared->num_initialized) {
|
|
|
|
// Other threads have finished
|
|
|
|
break;
|
2011-09-01 21:08:02 +02:00
|
|
|
}
|
2014-04-10 19:15:59 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
GenerateKeyFromInt(thread->rand.Next() % FLAGS_num, FLAGS_num, &key);
|
2015-03-18 21:50:52 +01:00
|
|
|
Status s;
|
|
|
|
|
|
|
|
if (write_merge == kPut) {
|
|
|
|
s = db->Put(write_options_, key, gen.Generate(value_size_));
|
|
|
|
} else {
|
|
|
|
s = db->Merge(write_options_, key, gen.Generate(value_size_));
|
|
|
|
}
|
|
|
|
|
2014-04-10 19:15:59 +02:00
|
|
|
if (!s.ok()) {
|
2015-03-18 21:50:52 +01:00
|
|
|
fprintf(stderr, "put or merge error: %s\n", s.ToString().c_str());
|
2014-04-10 19:15:59 +02:00
|
|
|
exit(1);
|
|
|
|
}
|
2015-03-30 20:28:25 +02:00
|
|
|
bytes += key.size() + value_size_;
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(&db_, db_.db, 1);
|
2013-04-17 19:56:39 +02:00
|
|
|
|
2014-04-10 19:15:59 +02:00
|
|
|
++num_writes;
|
|
|
|
if (writes_per_second_by_10 && num_writes >= writes_per_second_by_10) {
|
|
|
|
double now = FLAGS_env->NowMicros();
|
|
|
|
double usecs_since_last = now - last;
|
2013-04-17 19:56:39 +02:00
|
|
|
|
2014-04-10 19:15:59 +02:00
|
|
|
num_writes = 0;
|
|
|
|
last = now;
|
2013-04-17 19:56:39 +02:00
|
|
|
|
2014-04-10 19:15:59 +02:00
|
|
|
if (usecs_since_last < 100000.0) {
|
|
|
|
FLAGS_env->SleepForMicroseconds(100000.0 - usecs_since_last);
|
|
|
|
last = FLAGS_env->NowMicros();
|
2013-04-17 19:56:39 +02:00
|
|
|
}
|
2011-09-01 21:08:02 +02:00
|
|
|
}
|
|
|
|
}
|
2015-03-30 20:28:25 +02:00
|
|
|
thread->stats.AddBytes(bytes);
|
2011-09-01 21:08:02 +02:00
|
|
|
}
|
|
|
|
|
2013-02-20 02:51:06 +01:00
|
|
|
// Given a key K and value V, this puts (K+"0", V), (K+"1", V), (K+"2", V)
|
2013-06-12 21:42:21 +02:00
|
|
|
// in DB atomically i.e in a single batch. Also refer GetMany.
|
2014-04-11 21:15:09 +02:00
|
|
|
Status PutMany(DB* db, const WriteOptions& writeoptions, const Slice& key,
|
|
|
|
const Slice& value) {
|
2013-02-20 02:51:06 +01:00
|
|
|
std::string suffixes[3] = {"2", "1", "0"};
|
|
|
|
std::string keys[3];
|
|
|
|
|
|
|
|
WriteBatch batch;
|
|
|
|
Status s;
|
|
|
|
for (int i = 0; i < 3; i++) {
|
|
|
|
keys[i] = key.ToString() + suffixes[i];
|
|
|
|
batch.Put(keys[i], value);
|
|
|
|
}
|
|
|
|
|
2014-04-11 21:15:09 +02:00
|
|
|
s = db->Write(writeoptions, &batch);
|
2013-02-20 02:51:06 +01:00
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
// Given a key K, this deletes (K+"0", V), (K+"1", V), (K+"2", V)
|
2013-06-12 21:42:21 +02:00
|
|
|
// in DB atomically i.e in a single batch. Also refer GetMany.
|
2014-04-11 21:15:09 +02:00
|
|
|
Status DeleteMany(DB* db, const WriteOptions& writeoptions,
|
|
|
|
const Slice& key) {
|
2013-02-20 02:51:06 +01:00
|
|
|
std::string suffixes[3] = {"1", "2", "0"};
|
|
|
|
std::string keys[3];
|
|
|
|
|
|
|
|
WriteBatch batch;
|
|
|
|
Status s;
|
|
|
|
for (int i = 0; i < 3; i++) {
|
|
|
|
keys[i] = key.ToString() + suffixes[i];
|
|
|
|
batch.Delete(keys[i]);
|
|
|
|
}
|
|
|
|
|
2014-04-11 21:15:09 +02:00
|
|
|
s = db->Write(writeoptions, &batch);
|
2013-02-20 02:51:06 +01:00
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
// Given a key K and value V, this gets values for K+"0", K+"1" and K+"2"
|
|
|
|
// in the same snapshot, and verifies that all the values are identical.
|
2013-06-12 21:42:21 +02:00
|
|
|
// ASSUMES that PutMany was used to put (K, V) into the DB.
|
2014-04-11 21:15:09 +02:00
|
|
|
Status GetMany(DB* db, const ReadOptions& readoptions, const Slice& key,
|
|
|
|
std::string* value) {
|
2013-02-20 02:51:06 +01:00
|
|
|
std::string suffixes[3] = {"0", "1", "2"};
|
|
|
|
std::string keys[3];
|
|
|
|
Slice key_slices[3];
|
|
|
|
std::string values[3];
|
|
|
|
ReadOptions readoptionscopy = readoptions;
|
2014-04-11 21:15:09 +02:00
|
|
|
readoptionscopy.snapshot = db->GetSnapshot();
|
2013-02-20 02:51:06 +01:00
|
|
|
Status s;
|
|
|
|
for (int i = 0; i < 3; i++) {
|
|
|
|
keys[i] = key.ToString() + suffixes[i];
|
|
|
|
key_slices[i] = keys[i];
|
2014-04-11 21:15:09 +02:00
|
|
|
s = db->Get(readoptionscopy, key_slices[i], value);
|
2013-02-20 02:51:06 +01:00
|
|
|
if (!s.ok() && !s.IsNotFound()) {
|
|
|
|
fprintf(stderr, "get error: %s\n", s.ToString().c_str());
|
|
|
|
values[i] = "";
|
|
|
|
// we continue after error rather than exiting so that we can
|
|
|
|
// find more errors if any
|
|
|
|
} else if (s.IsNotFound()) {
|
|
|
|
values[i] = "";
|
|
|
|
} else {
|
|
|
|
values[i] = *value;
|
|
|
|
}
|
|
|
|
}
|
2014-04-11 21:15:09 +02:00
|
|
|
db->ReleaseSnapshot(readoptionscopy.snapshot);
|
2013-02-20 02:51:06 +01:00
|
|
|
|
|
|
|
if ((values[0] != values[1]) || (values[1] != values[2])) {
|
|
|
|
fprintf(stderr, "inconsistent values for key %s: %s, %s, %s\n",
|
|
|
|
key.ToString().c_str(), values[0].c_str(), values[1].c_str(),
|
|
|
|
values[2].c_str());
|
|
|
|
// we continue after error rather than exiting so that we can
|
|
|
|
// find more errors if any
|
|
|
|
}
|
|
|
|
|
|
|
|
return s;
|
|
|
|
}
|
|
|
|
|
|
|
|
// Differs from readrandomwriterandom in the following ways:
|
2013-06-12 21:42:21 +02:00
|
|
|
// (a) Uses GetMany/PutMany to read/write key values. Refer to those funcs.
|
2013-02-20 02:51:06 +01:00
|
|
|
// (b) Does deletes as well (per FLAGS_deletepercent)
|
|
|
|
// (c) In order to achieve high % of 'found' during lookups, and to do
|
|
|
|
// multiple writes (including puts and deletes) it uses upto
|
|
|
|
// FLAGS_numdistinct distinct keys instead of FLAGS_num distinct keys.
|
2013-06-12 21:42:21 +02:00
|
|
|
// (d) Does not have a MultiGet option.
|
2013-02-20 02:51:06 +01:00
|
|
|
void RandomWithVerify(ThreadState* thread) {
|
|
|
|
ReadOptions options(FLAGS_verify_checksum, true);
|
|
|
|
RandomGenerator gen;
|
|
|
|
std::string value;
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t found = 0;
|
2013-02-20 02:51:06 +01:00
|
|
|
int get_weight = 0;
|
|
|
|
int put_weight = 0;
|
|
|
|
int delete_weight = 0;
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t gets_done = 0;
|
|
|
|
int64_t puts_done = 0;
|
|
|
|
int64_t deletes_done = 0;
|
2013-06-12 21:42:21 +02:00
|
|
|
|
2015-02-19 23:27:48 +01:00
|
|
|
std::unique_ptr<const char[]> key_guard;
|
|
|
|
Slice key = AllocateKey(&key_guard);
|
2014-04-08 20:21:09 +02:00
|
|
|
|
2013-02-20 02:51:06 +01:00
|
|
|
// the number of iterations is the larger of read_ or write_
|
2014-03-05 02:08:05 +01:00
|
|
|
for (int64_t i = 0; i < readwrites_; i++) {
|
2014-04-11 21:15:09 +02:00
|
|
|
DB* db = SelectDB(thread);
|
2013-02-20 02:51:06 +01:00
|
|
|
if (get_weight == 0 && put_weight == 0 && delete_weight == 0) {
|
2013-06-12 21:42:21 +02:00
|
|
|
// one batch completed, reinitialize for next batch
|
2013-02-20 02:51:06 +01:00
|
|
|
get_weight = FLAGS_readwritepercent;
|
|
|
|
delete_weight = FLAGS_deletepercent;
|
|
|
|
put_weight = 100 - get_weight - delete_weight;
|
|
|
|
}
|
2014-04-08 20:21:09 +02:00
|
|
|
GenerateKeyFromInt(thread->rand.Next() % FLAGS_numdistinct,
|
|
|
|
FLAGS_numdistinct, &key);
|
2013-02-20 02:51:06 +01:00
|
|
|
if (get_weight > 0) {
|
|
|
|
// do all the gets first
|
2014-04-11 21:15:09 +02:00
|
|
|
Status s = GetMany(db, options, key, &value);
|
2013-02-20 02:51:06 +01:00
|
|
|
if (!s.ok() && !s.IsNotFound()) {
|
2013-06-12 21:42:21 +02:00
|
|
|
fprintf(stderr, "getmany error: %s\n", s.ToString().c_str());
|
2013-02-20 02:51:06 +01:00
|
|
|
// we continue after error rather than exiting so that we can
|
|
|
|
// find more errors if any
|
|
|
|
} else if (!s.IsNotFound()) {
|
|
|
|
found++;
|
|
|
|
}
|
|
|
|
get_weight--;
|
|
|
|
gets_done++;
|
|
|
|
} else if (put_weight > 0) {
|
|
|
|
// then do all the corresponding number of puts
|
|
|
|
// for all the gets we have done earlier
|
2014-04-11 21:15:09 +02:00
|
|
|
Status s = PutMany(db, write_options_, key, gen.Generate(value_size_));
|
2013-02-20 02:51:06 +01:00
|
|
|
if (!s.ok()) {
|
2013-06-12 21:42:21 +02:00
|
|
|
fprintf(stderr, "putmany error: %s\n", s.ToString().c_str());
|
2013-02-20 02:51:06 +01:00
|
|
|
exit(1);
|
|
|
|
}
|
|
|
|
put_weight--;
|
|
|
|
puts_done++;
|
|
|
|
} else if (delete_weight > 0) {
|
2014-04-11 21:15:09 +02:00
|
|
|
Status s = DeleteMany(db, write_options_, key);
|
2013-02-20 02:51:06 +01:00
|
|
|
if (!s.ok()) {
|
2013-06-12 21:42:21 +02:00
|
|
|
fprintf(stderr, "deletemany error: %s\n", s.ToString().c_str());
|
2013-02-20 02:51:06 +01:00
|
|
|
exit(1);
|
|
|
|
}
|
|
|
|
delete_weight--;
|
|
|
|
deletes_done++;
|
|
|
|
}
|
|
|
|
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(&db_, db_.db, 1);
|
2013-02-20 02:51:06 +01:00
|
|
|
}
|
|
|
|
char msg[100];
|
2013-08-23 07:37:13 +02:00
|
|
|
snprintf(msg, sizeof(msg),
|
2014-03-05 02:08:05 +01:00
|
|
|
"( get:%" PRIu64 " put:%" PRIu64 " del:%" PRIu64 " total:%" \
|
|
|
|
PRIu64 " found:%" PRIu64 ")",
|
2013-02-20 02:51:06 +01:00
|
|
|
gets_done, puts_done, deletes_done, readwrites_, found);
|
|
|
|
thread->stats.AddMessage(msg);
|
|
|
|
}
|
|
|
|
|
2013-08-13 22:58:02 +02:00
|
|
|
// This is different from ReadWhileWriting because it does not use
|
2012-09-26 17:06:19 +02:00
|
|
|
// an extra thread.
|
2012-08-31 08:06:17 +02:00
|
|
|
void ReadRandomWriteRandom(ThreadState* thread) {
|
|
|
|
ReadOptions options(FLAGS_verify_checksum, true);
|
|
|
|
RandomGenerator gen;
|
|
|
|
std::string value;
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t found = 0;
|
2012-08-31 08:06:17 +02:00
|
|
|
int get_weight = 0;
|
|
|
|
int put_weight = 0;
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t reads_done = 0;
|
|
|
|
int64_t writes_done = 0;
|
2013-03-01 20:25:13 +01:00
|
|
|
Duration duration(FLAGS_duration, readwrites_);
|
|
|
|
|
2015-02-19 23:27:48 +01:00
|
|
|
std::unique_ptr<const char[]> key_guard;
|
|
|
|
Slice key = AllocateKey(&key_guard);
|
2014-04-08 20:21:09 +02:00
|
|
|
|
2012-08-31 08:06:17 +02:00
|
|
|
// the number of iterations is the larger of read_ or write_
|
2013-03-01 20:25:13 +01:00
|
|
|
while (!duration.Done(1)) {
|
2014-04-11 21:15:09 +02:00
|
|
|
DB* db = SelectDB(thread);
|
2014-04-08 20:21:09 +02:00
|
|
|
GenerateKeyFromInt(thread->rand.Next() % FLAGS_num, FLAGS_num, &key);
|
2012-08-31 08:06:17 +02:00
|
|
|
if (get_weight == 0 && put_weight == 0) {
|
2013-08-13 22:58:02 +02:00
|
|
|
// one batch completed, reinitialize for next batch
|
2012-08-31 08:06:17 +02:00
|
|
|
get_weight = FLAGS_readwritepercent;
|
|
|
|
put_weight = 100 - get_weight;
|
|
|
|
}
|
|
|
|
if (get_weight > 0) {
|
|
|
|
// do all the gets first
|
2014-04-11 21:15:09 +02:00
|
|
|
Status s = db->Get(options, key, &value);
|
2013-02-20 02:51:06 +01:00
|
|
|
if (!s.ok() && !s.IsNotFound()) {
|
|
|
|
fprintf(stderr, "get error: %s\n", s.ToString().c_str());
|
|
|
|
// we continue after error rather than exiting so that we can
|
|
|
|
// find more errors if any
|
|
|
|
} else if (!s.IsNotFound()) {
|
|
|
|
found++;
|
|
|
|
}
|
2012-08-31 08:06:17 +02:00
|
|
|
get_weight--;
|
|
|
|
reads_done++;
|
|
|
|
} else if (put_weight > 0) {
|
|
|
|
// then do all the corresponding number of puts
|
|
|
|
// for all the gets we have done earlier
|
2014-04-11 21:15:09 +02:00
|
|
|
Status s = db->Put(write_options_, key, gen.Generate(value_size_));
|
2012-08-31 08:06:17 +02:00
|
|
|
if (!s.ok()) {
|
|
|
|
fprintf(stderr, "put error: %s\n", s.ToString().c_str());
|
|
|
|
exit(1);
|
|
|
|
}
|
|
|
|
put_weight--;
|
|
|
|
writes_done++;
|
|
|
|
}
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(nullptr, db, 1);
|
2012-08-31 08:06:17 +02:00
|
|
|
}
|
|
|
|
char msg[100];
|
2014-03-05 02:08:05 +01:00
|
|
|
snprintf(msg, sizeof(msg), "( reads:%" PRIu64 " writes:%" PRIu64 \
|
|
|
|
" total:%" PRIu64 " found:%" PRIu64 ")",
|
2013-02-20 02:51:06 +01:00
|
|
|
reads_done, writes_done, readwrites_, found);
|
2012-08-31 08:06:17 +02:00
|
|
|
thread->stats.AddMessage(msg);
|
|
|
|
}
|
|
|
|
|
2013-03-01 20:25:13 +01:00
|
|
|
//
|
|
|
|
// Read-modify-write for random keys
|
|
|
|
void UpdateRandom(ThreadState* thread) {
|
|
|
|
ReadOptions options(FLAGS_verify_checksum, true);
|
|
|
|
RandomGenerator gen;
|
|
|
|
std::string value;
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t found = 0;
|
2015-03-30 20:28:25 +02:00
|
|
|
int64_t bytes = 0;
|
2013-03-01 20:25:13 +01:00
|
|
|
Duration duration(FLAGS_duration, readwrites_);
|
|
|
|
|
2015-02-19 23:27:48 +01:00
|
|
|
std::unique_ptr<const char[]> key_guard;
|
|
|
|
Slice key = AllocateKey(&key_guard);
|
2013-03-01 20:25:13 +01:00
|
|
|
// the number of iterations is the larger of read_ or write_
|
|
|
|
while (!duration.Done(1)) {
|
2014-04-11 21:15:09 +02:00
|
|
|
DB* db = SelectDB(thread);
|
2014-04-08 20:21:09 +02:00
|
|
|
GenerateKeyFromInt(thread->rand.Next() % FLAGS_num, FLAGS_num, &key);
|
2013-03-01 20:25:13 +01:00
|
|
|
|
2015-01-22 03:18:15 +01:00
|
|
|
auto status = db->Get(options, key, &value);
|
|
|
|
if (status.ok()) {
|
|
|
|
++found;
|
2015-03-30 20:28:25 +02:00
|
|
|
bytes += key.size() + value.size();
|
2015-01-22 03:18:15 +01:00
|
|
|
} else if (!status.IsNotFound()) {
|
2015-01-22 03:23:12 +01:00
|
|
|
fprintf(stderr, "Get returned an error: %s\n",
|
|
|
|
status.ToString().c_str());
|
2015-01-22 03:18:15 +01:00
|
|
|
abort();
|
2013-03-01 20:25:13 +01:00
|
|
|
}
|
|
|
|
|
2014-04-11 21:15:09 +02:00
|
|
|
Status s = db->Put(write_options_, key, gen.Generate(value_size_));
|
2013-03-01 20:25:13 +01:00
|
|
|
if (!s.ok()) {
|
|
|
|
fprintf(stderr, "put error: %s\n", s.ToString().c_str());
|
|
|
|
exit(1);
|
|
|
|
}
|
2015-03-30 20:28:25 +02:00
|
|
|
bytes += key.size() + value_size_;
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(nullptr, db, 1);
|
2013-03-01 20:25:13 +01:00
|
|
|
}
|
|
|
|
char msg[100];
|
2013-08-23 07:37:13 +02:00
|
|
|
snprintf(msg, sizeof(msg),
|
2014-03-05 02:08:05 +01:00
|
|
|
"( updates:%" PRIu64 " found:%" PRIu64 ")", readwrites_, found);
|
2015-03-30 20:28:25 +02:00
|
|
|
thread->stats.AddBytes(bytes);
|
2013-03-01 20:25:13 +01:00
|
|
|
thread->stats.AddMessage(msg);
|
|
|
|
}
|
|
|
|
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
// Read-modify-write for random keys.
|
|
|
|
// Each operation causes the key grow by value_size (simulating an append).
|
|
|
|
// Generally used for benchmarking against merges of similar type
|
|
|
|
void AppendRandom(ThreadState* thread) {
|
|
|
|
ReadOptions options(FLAGS_verify_checksum, true);
|
|
|
|
RandomGenerator gen;
|
|
|
|
std::string value;
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t found = 0;
|
2015-03-30 20:28:25 +02:00
|
|
|
int64_t bytes = 0;
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
|
2015-02-19 23:27:48 +01:00
|
|
|
std::unique_ptr<const char[]> key_guard;
|
|
|
|
Slice key = AllocateKey(&key_guard);
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
// The number of iterations is the larger of read_ or write_
|
|
|
|
Duration duration(FLAGS_duration, readwrites_);
|
|
|
|
while (!duration.Done(1)) {
|
2014-04-11 21:15:09 +02:00
|
|
|
DB* db = SelectDB(thread);
|
2014-04-08 20:21:09 +02:00
|
|
|
GenerateKeyFromInt(thread->rand.Next() % FLAGS_num, FLAGS_num, &key);
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
|
2015-01-22 03:18:15 +01:00
|
|
|
auto status = db->Get(options, key, &value);
|
|
|
|
if (status.ok()) {
|
|
|
|
++found;
|
2015-03-30 20:28:25 +02:00
|
|
|
bytes += key.size() + value.size();
|
2015-01-22 03:18:15 +01:00
|
|
|
} else if (!status.IsNotFound()) {
|
2015-01-22 03:23:12 +01:00
|
|
|
fprintf(stderr, "Get returned an error: %s\n",
|
|
|
|
status.ToString().c_str());
|
2015-01-22 03:18:15 +01:00
|
|
|
abort();
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
} else {
|
|
|
|
// If not existing, then just assume an empty string of data
|
|
|
|
value.clear();
|
|
|
|
}
|
|
|
|
|
|
|
|
// Update the value (by appending data)
|
|
|
|
Slice operand = gen.Generate(value_size_);
|
|
|
|
if (value.size() > 0) {
|
2015-08-11 20:46:15 +02:00
|
|
|
// Use a delimiter to match the semantics for StringAppendOperator
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
value.append(1,',');
|
|
|
|
}
|
|
|
|
value.append(operand.data(), operand.size());
|
|
|
|
|
|
|
|
// Write back to the database
|
2014-04-11 21:15:09 +02:00
|
|
|
Status s = db->Put(write_options_, key, value);
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
if (!s.ok()) {
|
|
|
|
fprintf(stderr, "put error: %s\n", s.ToString().c_str());
|
|
|
|
exit(1);
|
|
|
|
}
|
2015-03-30 20:28:25 +02:00
|
|
|
bytes += key.size() + value.size();
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(nullptr, db, 1);
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
}
|
2014-04-08 20:21:09 +02:00
|
|
|
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
char msg[100];
|
2014-03-05 02:08:05 +01:00
|
|
|
snprintf(msg, sizeof(msg), "( updates:%" PRIu64 " found:%" PRIu64 ")",
|
|
|
|
readwrites_, found);
|
2015-03-30 20:28:25 +02:00
|
|
|
thread->stats.AddBytes(bytes);
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
thread->stats.AddMessage(msg);
|
|
|
|
}
|
|
|
|
|
|
|
|
// Read-modify-write for random keys (using MergeOperator)
|
|
|
|
// The merge operator to use should be defined by FLAGS_merge_operator
|
|
|
|
// Adjust FLAGS_value_size so that the keys are reasonable for this operator
|
|
|
|
// Assumes that the merge operator is non-null (i.e.: is well-defined)
|
|
|
|
//
|
|
|
|
// For example, use FLAGS_merge_operator="uint64add" and FLAGS_value_size=8
|
|
|
|
// to simulate random additions over 64-bit integers using merge.
|
2014-01-11 02:33:56 +01:00
|
|
|
//
|
|
|
|
// The number of merges on the same key can be controlled by adjusting
|
|
|
|
// FLAGS_merge_keys.
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
void MergeRandom(ThreadState* thread) {
|
|
|
|
RandomGenerator gen;
|
2015-03-30 20:28:25 +02:00
|
|
|
int64_t bytes = 0;
|
2015-02-19 23:27:48 +01:00
|
|
|
std::unique_ptr<const char[]> key_guard;
|
|
|
|
Slice key = AllocateKey(&key_guard);
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
// The number of iterations is the larger of read_ or write_
|
|
|
|
Duration duration(FLAGS_duration, readwrites_);
|
|
|
|
while (!duration.Done(1)) {
|
2014-04-11 21:15:09 +02:00
|
|
|
DB* db = SelectDB(thread);
|
2014-04-08 20:21:09 +02:00
|
|
|
GenerateKeyFromInt(thread->rand.Next() % merge_keys_, merge_keys_, &key);
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
|
2014-04-11 21:15:09 +02:00
|
|
|
Status s = db->Merge(write_options_, key, gen.Generate(value_size_));
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
|
|
|
|
if (!s.ok()) {
|
|
|
|
fprintf(stderr, "merge error: %s\n", s.ToString().c_str());
|
|
|
|
exit(1);
|
|
|
|
}
|
2015-03-30 20:28:25 +02:00
|
|
|
bytes += key.size() + value_size_;
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(nullptr, db, 1);
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
// Print some statistics
|
|
|
|
char msg[100];
|
2014-03-05 02:08:05 +01:00
|
|
|
snprintf(msg, sizeof(msg), "( updates:%" PRIu64 ")", readwrites_);
|
2015-03-30 20:28:25 +02:00
|
|
|
thread->stats.AddBytes(bytes);
|
Benchmarking for Merge Operator
Summary:
Updated db_bench and utilities/merge_operators.h to allow for dynamic benchmarking
of merge operators in db_bench. Added a new test (--benchmarks=mergerandom), which performs
a bunch of random Merge() operations over random keys. Also added a "--merge_operator=" flag
so that the tester can easily benchmark different merge operators. Currently supports
the PutOperator and UInt64Add operator. Support for stringappend or list append may come later.
Test Plan:
1. make db_bench
2. Test the PutOperator (simulating Put) as follows:
./db_bench --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom --merge_operator=put
--threads=2
3. Test the UInt64AddOperator (simulating numeric addition) similarly:
./db_bench --value_size=8 --benchmarks=fillrandom,readrandom,updaterandom,readrandom,mergerandom,readrandom
--merge_operator=uint64add --threads=2
Reviewers: haobo, dhruba, zshao, MarkCallaghan
Reviewed By: haobo
CC: leveldb
Differential Revision: https://reviews.facebook.net/D11535
2013-08-16 02:13:07 +02:00
|
|
|
thread->stats.AddMessage(msg);
|
|
|
|
}
|
|
|
|
|
2014-01-11 02:33:56 +01:00
|
|
|
// Read and merge random keys. The amount of reads and merges are controlled
|
|
|
|
// by adjusting FLAGS_num and FLAGS_mergereadpercent. The number of distinct
|
|
|
|
// keys (and thus also the number of reads and merges on the same key) can be
|
|
|
|
// adjusted with FLAGS_merge_keys.
|
|
|
|
//
|
|
|
|
// As with MergeRandom, the merge operator to use should be defined by
|
|
|
|
// FLAGS_merge_operator.
|
|
|
|
void ReadRandomMergeRandom(ThreadState* thread) {
|
|
|
|
ReadOptions options(FLAGS_verify_checksum, true);
|
|
|
|
RandomGenerator gen;
|
|
|
|
std::string value;
|
2014-03-05 02:08:05 +01:00
|
|
|
int64_t num_hits = 0;
|
|
|
|
int64_t num_gets = 0;
|
|
|
|
int64_t num_merges = 0;
|
2014-01-11 02:33:56 +01:00
|
|
|
size_t max_length = 0;
|
|
|
|
|
2015-02-19 23:27:48 +01:00
|
|
|
std::unique_ptr<const char[]> key_guard;
|
|
|
|
Slice key = AllocateKey(&key_guard);
|
2014-01-11 02:33:56 +01:00
|
|
|
// the number of iterations is the larger of read_ or write_
|
|
|
|
Duration duration(FLAGS_duration, readwrites_);
|
|
|
|
while (!duration.Done(1)) {
|
2014-04-11 21:15:09 +02:00
|
|
|
DB* db = SelectDB(thread);
|
2014-04-08 20:21:09 +02:00
|
|
|
GenerateKeyFromInt(thread->rand.Next() % merge_keys_, merge_keys_, &key);
|
2014-01-11 02:33:56 +01:00
|
|
|
|
|
|
|
bool do_merge = int(thread->rand.Next() % 100) < FLAGS_mergereadpercent;
|
|
|
|
|
|
|
|
if (do_merge) {
|
2014-04-11 21:15:09 +02:00
|
|
|
Status s = db->Merge(write_options_, key, gen.Generate(value_size_));
|
2014-01-11 02:33:56 +01:00
|
|
|
if (!s.ok()) {
|
|
|
|
fprintf(stderr, "merge error: %s\n", s.ToString().c_str());
|
|
|
|
exit(1);
|
|
|
|
}
|
|
|
|
|
|
|
|
num_merges++;
|
|
|
|
|
|
|
|
} else {
|
2014-04-11 21:15:09 +02:00
|
|
|
Status s = db->Get(options, key, &value);
|
2014-01-11 02:33:56 +01:00
|
|
|
if (value.length() > max_length)
|
|
|
|
max_length = value.length();
|
|
|
|
|
|
|
|
if (!s.ok() && !s.IsNotFound()) {
|
|
|
|
fprintf(stderr, "get error: %s\n", s.ToString().c_str());
|
|
|
|
// we continue after error rather than exiting so that we can
|
|
|
|
// find more errors if any
|
|
|
|
} else if (!s.IsNotFound()) {
|
|
|
|
num_hits++;
|
|
|
|
}
|
|
|
|
|
|
|
|
num_gets++;
|
|
|
|
|
|
|
|
}
|
|
|
|
|
2014-09-29 18:50:41 +02:00
|
|
|
thread->stats.FinishedOps(nullptr, db, 1);
|
2014-01-11 02:33:56 +01:00
|
|
|
}
|
2014-04-08 20:21:09 +02:00
|
|
|
|
2014-01-11 02:33:56 +01:00
|
|
|
char msg[100];
|
|
|
|
snprintf(msg, sizeof(msg),
|
2015-07-13 21:11:05 +02:00
|
|
|
"(reads:%" PRIu64 " merges:%" PRIu64 " total:%" PRIu64
|
|
|
|
" hits:%" PRIu64 " maxlength:%" ROCKSDB_PRIszt ")",
|
2014-01-11 02:33:56 +01:00
|
|
|
num_gets, num_merges, readwrites_, num_hits, max_length);
|
|
|
|
thread->stats.AddMessage(msg);
|
|
|
|
}
|
|
|
|
|
SkipListRep::LookaheadIterator
Summary:
This diff introduces the `lookahead` argument to `SkipListFactory()`. This is an
optimization for the tailing use case which includes many seeks. E.g. consider
the following operations on a skip list iterator:
Seek(x), Next(), Next(), Seek(x+2), Next(), Seek(x+3), Next(), Next(), ...
If `lookahead` is positive, `SkipListRep` will return an iterator which also
keeps track of the previously visited node. Seek() then first does a linear
search starting from that node (up to `lookahead` steps). As in the tailing
example above, this may require fewer than ~log(n) comparisons as with regular
skip list search.
Test Plan:
Added a new benchmark (`fillseekseq`) which simulates the usage pattern. It
first writes N records (with consecutive keys), then measures how much time it
takes to read them by calling `Seek()` and `Next()`.
$ time ./db_bench -num 10000000 -benchmarks fillseekseq -prefix_size 1 \
-key_size 8 -write_buffer_size $[1024*1024*1024] -value_size 50 \
-seekseq_next 2 -skip_list_lookahead=0
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.389 micros/op 2569047 ops/sec;
real 0m21.806s
user 0m12.106s
sys 0m9.672s
$ time ./db_bench [...] -skip_list_lookahead=2
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.153 micros/op 6540684 ops/sec;
real 0m19.469s
user 0m10.192s
sys 0m9.252s
Reviewers: ljin, sdong, igor
Reviewed By: igor
Subscribers: dhruba, leveldb, march, lovro
Differential Revision: https://reviews.facebook.net/D23997
2014-09-24 00:52:28 +02:00
|
|
|
void WriteSeqSeekSeq(ThreadState* thread) {
|
|
|
|
writes_ = FLAGS_num;
|
|
|
|
DoWrite(thread, SEQUENTIAL);
|
|
|
|
// exclude writes from the ops/sec calculation
|
|
|
|
thread->stats.Start(thread->tid);
|
|
|
|
|
|
|
|
DB* db = SelectDB(thread);
|
|
|
|
std::unique_ptr<Iterator> iter(
|
|
|
|
db->NewIterator(ReadOptions(FLAGS_verify_checksum, true)));
|
|
|
|
|
2015-02-19 23:27:48 +01:00
|
|
|
std::unique_ptr<const char[]> key_guard;
|
|
|
|
Slice key = AllocateKey(&key_guard);
|
SkipListRep::LookaheadIterator
Summary:
This diff introduces the `lookahead` argument to `SkipListFactory()`. This is an
optimization for the tailing use case which includes many seeks. E.g. consider
the following operations on a skip list iterator:
Seek(x), Next(), Next(), Seek(x+2), Next(), Seek(x+3), Next(), Next(), ...
If `lookahead` is positive, `SkipListRep` will return an iterator which also
keeps track of the previously visited node. Seek() then first does a linear
search starting from that node (up to `lookahead` steps). As in the tailing
example above, this may require fewer than ~log(n) comparisons as with regular
skip list search.
Test Plan:
Added a new benchmark (`fillseekseq`) which simulates the usage pattern. It
first writes N records (with consecutive keys), then measures how much time it
takes to read them by calling `Seek()` and `Next()`.
$ time ./db_bench -num 10000000 -benchmarks fillseekseq -prefix_size 1 \
-key_size 8 -write_buffer_size $[1024*1024*1024] -value_size 50 \
-seekseq_next 2 -skip_list_lookahead=0
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.389 micros/op 2569047 ops/sec;
real 0m21.806s
user 0m12.106s
sys 0m9.672s
$ time ./db_bench [...] -skip_list_lookahead=2
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.153 micros/op 6540684 ops/sec;
real 0m19.469s
user 0m10.192s
sys 0m9.252s
Reviewers: ljin, sdong, igor
Reviewed By: igor
Subscribers: dhruba, leveldb, march, lovro
Differential Revision: https://reviews.facebook.net/D23997
2014-09-24 00:52:28 +02:00
|
|
|
for (int64_t i = 0; i < FLAGS_num; ++i) {
|
|
|
|
GenerateKeyFromInt(i, FLAGS_num, &key);
|
|
|
|
iter->Seek(key);
|
|
|
|
assert(iter->Valid() && iter->key() == key);
|
|
|
|
thread->stats.FinishedOps(nullptr, db, 1);
|
|
|
|
|
2014-10-17 19:20:25 +02:00
|
|
|
for (int j = 0; j < FLAGS_seek_nexts && i + 1 < FLAGS_num; ++j) {
|
2015-03-18 21:50:52 +01:00
|
|
|
if (!FLAGS_reverse_iterator) {
|
|
|
|
iter->Next();
|
|
|
|
} else {
|
|
|
|
iter->Prev();
|
|
|
|
}
|
SkipListRep::LookaheadIterator
Summary:
This diff introduces the `lookahead` argument to `SkipListFactory()`. This is an
optimization for the tailing use case which includes many seeks. E.g. consider
the following operations on a skip list iterator:
Seek(x), Next(), Next(), Seek(x+2), Next(), Seek(x+3), Next(), Next(), ...
If `lookahead` is positive, `SkipListRep` will return an iterator which also
keeps track of the previously visited node. Seek() then first does a linear
search starting from that node (up to `lookahead` steps). As in the tailing
example above, this may require fewer than ~log(n) comparisons as with regular
skip list search.
Test Plan:
Added a new benchmark (`fillseekseq`) which simulates the usage pattern. It
first writes N records (with consecutive keys), then measures how much time it
takes to read them by calling `Seek()` and `Next()`.
$ time ./db_bench -num 10000000 -benchmarks fillseekseq -prefix_size 1 \
-key_size 8 -write_buffer_size $[1024*1024*1024] -value_size 50 \
-seekseq_next 2 -skip_list_lookahead=0
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.389 micros/op 2569047 ops/sec;
real 0m21.806s
user 0m12.106s
sys 0m9.672s
$ time ./db_bench [...] -skip_list_lookahead=2
[...]
DB path: [/dev/shm/rocksdbtest/dbbench]
fillseekseq : 0.153 micros/op 6540684 ops/sec;
real 0m19.469s
user 0m10.192s
sys 0m9.252s
Reviewers: ljin, sdong, igor
Reviewed By: igor
Subscribers: dhruba, leveldb, march, lovro
Differential Revision: https://reviews.facebook.net/D23997
2014-09-24 00:52:28 +02:00
|
|
|
GenerateKeyFromInt(++i, FLAGS_num, &key);
|
|
|
|
assert(iter->Valid() && iter->key() == key);
|
|
|
|
thread->stats.FinishedOps(nullptr, db, 1);
|
|
|
|
}
|
|
|
|
|
|
|
|
iter->Seek(key);
|
|
|
|
assert(iter->Valid() && iter->key() == key);
|
|
|
|
thread->stats.FinishedOps(nullptr, db, 1);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-10-14 21:43:00 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
2015-05-29 23:36:35 +02:00
|
|
|
// This benchmark stress tests Transactions. For a given --duration (or
|
|
|
|
// total number of --writes, a Transaction will perform a read-modify-write
|
|
|
|
// to increment the value of a key in each of N(--transaction-sets) sets of
|
|
|
|
// keys (where each set has --num keys). If --threads is set, this will be
|
|
|
|
// done in parallel.
|
|
|
|
//
|
|
|
|
// To test transactions, use --transaction_db=true. Not setting this
|
|
|
|
// parameter
|
|
|
|
// will run the same benchmark without transactions.
|
|
|
|
//
|
|
|
|
// RandomTransactionVerify() will then validate the correctness of the results
|
|
|
|
// by checking if the sum of all keys in each set is the same.
|
|
|
|
void RandomTransaction(ThreadState* thread) {
|
|
|
|
ReadOptions options(FLAGS_verify_checksum, true);
|
|
|
|
Duration duration(FLAGS_duration, readwrites_);
|
|
|
|
ReadOptions read_options(FLAGS_verify_checksum, true);
|
|
|
|
std::string value;
|
|
|
|
DB* db = db_.db;
|
|
|
|
uint64_t transactions_done = 0;
|
|
|
|
uint64_t transactions_aborted = 0;
|
|
|
|
Status s;
|
|
|
|
uint64_t num_prefix_ranges = FLAGS_transaction_sets;
|
|
|
|
|
|
|
|
if (num_prefix_ranges == 0 || num_prefix_ranges > 9999) {
|
|
|
|
fprintf(stderr, "invalid value for transaction_sets\n");
|
|
|
|
abort();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (FLAGS_num_multi_db > 1) {
|
|
|
|
fprintf(stderr,
|
|
|
|
"Cannot run RandomTransaction benchmark with "
|
|
|
|
"FLAGS_multi_db > 1.");
|
|
|
|
abort();
|
|
|
|
}
|
|
|
|
|
|
|
|
while (!duration.Done(1)) {
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
Transaction* txn = nullptr;
|
2015-05-29 23:36:35 +02:00
|
|
|
WriteBatch* batch = nullptr;
|
|
|
|
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
if (FLAGS_optimistic_transaction_db) {
|
|
|
|
txn = db_.opt_txn_db->BeginTransaction(write_options_);
|
2015-05-29 23:36:35 +02:00
|
|
|
assert(txn);
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
} else if (FLAGS_transaction_db) {
|
|
|
|
TransactionDB* txn_db = reinterpret_cast<TransactionDB*>(db_.db);
|
2015-08-29 01:09:34 +02:00
|
|
|
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
TransactionOptions txn_options;
|
2015-08-29 01:09:34 +02:00
|
|
|
txn_options.lock_timeout = FLAGS_transaction_lock_timeout;
|
|
|
|
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
txn = txn_db->BeginTransaction(write_options_, txn_options);
|
2015-08-29 01:09:34 +02:00
|
|
|
assert(txn);
|
2015-05-29 23:36:35 +02:00
|
|
|
} else {
|
|
|
|
batch = new WriteBatch();
|
|
|
|
}
|
|
|
|
|
2015-08-29 01:09:34 +02:00
|
|
|
if (txn && FLAGS_transaction_set_snapshot) {
|
|
|
|
txn->SetSnapshot();
|
|
|
|
}
|
|
|
|
|
2015-05-29 23:36:35 +02:00
|
|
|
// pick a random number to use to increment a key in each set
|
|
|
|
uint64_t incr = (thread->rand.Next() % 100) + 1;
|
|
|
|
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
bool failed = false;
|
2015-05-29 23:36:35 +02:00
|
|
|
// For each set, pick a key at random and increment it
|
|
|
|
for (uint8_t i = 0; i < num_prefix_ranges; i++) {
|
|
|
|
uint64_t int_value;
|
|
|
|
char prefix_buf[5];
|
|
|
|
|
|
|
|
// key format: [SET#][random#]
|
|
|
|
std::string rand_key = ToString(thread->rand.Next() % FLAGS_num);
|
|
|
|
Slice base_key(rand_key);
|
|
|
|
|
|
|
|
// Pad prefix appropriately so we can iterate over each set
|
|
|
|
snprintf(prefix_buf, sizeof(prefix_buf), "%04d", i + 1);
|
|
|
|
std::string full_key = std::string(prefix_buf) + base_key.ToString();
|
|
|
|
Slice key(full_key);
|
|
|
|
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
if (txn) {
|
|
|
|
s = txn->GetForUpdate(read_options, key, &value);
|
2015-05-29 23:36:35 +02:00
|
|
|
} else {
|
|
|
|
s = db->Get(read_options, key, &value);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (s.ok()) {
|
|
|
|
int_value = std::stoull(value);
|
|
|
|
|
|
|
|
if (int_value == 0 || int_value == ULONG_MAX) {
|
|
|
|
fprintf(stderr, "Get returned unexpected value: %s\n",
|
|
|
|
value.c_str());
|
|
|
|
abort();
|
|
|
|
}
|
|
|
|
} else if (s.IsNotFound()) {
|
|
|
|
int_value = 0;
|
2015-08-29 01:09:34 +02:00
|
|
|
} else if (!(s.IsBusy() || s.IsTimedOut() || s.IsTryAgain())) {
|
|
|
|
fprintf(stderr, "Get returned an unexpected error: %s\n",
|
|
|
|
s.ToString().c_str());
|
2015-05-29 23:36:35 +02:00
|
|
|
abort();
|
2015-08-29 01:09:34 +02:00
|
|
|
} else {
|
|
|
|
failed = true;
|
|
|
|
break;
|
2015-05-29 23:36:35 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
if (FLAGS_transaction_sleep > 0) {
|
|
|
|
FLAGS_env->SleepForMicroseconds(thread->rand.Next() %
|
|
|
|
FLAGS_transaction_sleep);
|
|
|
|
}
|
|
|
|
|
|
|
|
std::string sum = ToString(int_value + incr);
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
if (txn) {
|
|
|
|
s = txn->Put(key, sum);
|
|
|
|
if (!s.ok()) {
|
2015-08-29 01:09:34 +02:00
|
|
|
// Since we did a GetForUpdate, Put should not fail.
|
|
|
|
fprintf(stderr, "Put returned an unexpected error: %s\n",
|
|
|
|
s.ToString().c_str());
|
|
|
|
abort();
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
}
|
2015-05-29 23:36:35 +02:00
|
|
|
} else {
|
|
|
|
batch->Put(key, sum);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
if (txn) {
|
|
|
|
if (failed) {
|
2015-08-29 01:09:34 +02:00
|
|
|
transactions_aborted++;
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
txn->Rollback();
|
2015-08-29 01:09:34 +02:00
|
|
|
s = Status::OK();
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
} else {
|
|
|
|
s = txn->Commit();
|
|
|
|
}
|
2015-05-29 23:36:35 +02:00
|
|
|
} else {
|
|
|
|
s = db->Write(write_options_, batch);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!s.ok()) {
|
2015-08-29 01:09:34 +02:00
|
|
|
failed = true;
|
|
|
|
|
2015-05-29 23:36:35 +02:00
|
|
|
// Ideally, we'd want to run this stress test with enough concurrency
|
|
|
|
// on a small enough set of keys that we get some failed transactions
|
|
|
|
// due to conflicts.
|
2015-08-29 01:09:34 +02:00
|
|
|
if (FLAGS_optimistic_transaction_db &&
|
|
|
|
(s.IsBusy() || s.IsTimedOut() || s.IsTryAgain())) {
|
|
|
|
transactions_aborted++;
|
|
|
|
} else if (FLAGS_transaction_db && s.IsExpired()) {
|
2015-05-29 23:36:35 +02:00
|
|
|
transactions_aborted++;
|
|
|
|
} else {
|
|
|
|
fprintf(stderr, "Unexpected write error: %s\n", s.ToString().c_str());
|
|
|
|
abort();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (txn) {
|
|
|
|
delete txn;
|
|
|
|
}
|
|
|
|
if (batch) {
|
|
|
|
delete batch;
|
|
|
|
}
|
|
|
|
|
2015-08-29 01:09:34 +02:00
|
|
|
if (!failed) {
|
|
|
|
thread->stats.FinishedOps(nullptr, db, 1);
|
|
|
|
}
|
|
|
|
|
2015-05-29 23:36:35 +02:00
|
|
|
transactions_done++;
|
|
|
|
}
|
|
|
|
|
|
|
|
char msg[100];
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
if (FLAGS_optimistic_transaction_db || FLAGS_transaction_db) {
|
2015-05-29 23:36:35 +02:00
|
|
|
snprintf(msg, sizeof(msg),
|
|
|
|
"( transactions:%" PRIu64 " aborts:%" PRIu64 ")",
|
|
|
|
transactions_done, transactions_aborted);
|
|
|
|
} else {
|
|
|
|
snprintf(msg, sizeof(msg), "( batches:%" PRIu64 " )", transactions_done);
|
|
|
|
}
|
|
|
|
thread->stats.AddMessage(msg);
|
|
|
|
|
|
|
|
if (FLAGS_perf_level > 0) {
|
|
|
|
thread->stats.AddMessage(perf_context.ToString());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Verifies consistency of data after RandomTransaction() has been run.
|
|
|
|
// Since each iteration of RandomTransaction() incremented a key in each set
|
|
|
|
// by the same value, the sum of the keys in each set should be the same.
|
|
|
|
void RandomTransactionVerify() {
|
Pessimistic Transactions
Summary:
Initial implementation of Pessimistic Transactions. This diff contains the api changes discussed in D38913. This diff is pretty large, so let me know if people would prefer to meet up to discuss it.
MyRocks folks: please take a look at the API in include/rocksdb/utilities/transaction[_db].h and let me know if you have any issues.
Also, you'll notice a couple of TODOs in the implementation of RollbackToSavePoint(). After chatting with Siying, I'm going to send out a separate diff for an alternate implementation of this feature that implements the rollback inside of WriteBatch/WriteBatchWithIndex. We can then decide which route is preferable.
Next, I'm planning on doing some perf testing and then integrating this diff into MongoRocks for further testing.
Test Plan: Unit tests, db_bench parallel testing.
Reviewers: igor, rven, sdong, yhchiang, yoshinorim
Reviewed By: sdong
Subscribers: hermanlee4, maykov, spetrunia, leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D40869
2015-05-26 02:37:33 +02:00
|
|
|
if (!FLAGS_transaction_db && !FLAGS_optimistic_transaction_db) {
|
2015-05-29 23:36:35 +02:00
|
|
|
// transactions not used, nothing to verify.
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
uint64_t prev_total = 0;
|
|
|
|
|
|
|
|
// For each set of keys with the same prefix, sum all the values
|
|
|
|
for (uint32_t i = 0; i < FLAGS_transaction_sets; i++) {
|
|
|
|
char prefix_buf[5];
|
|
|
|
snprintf(prefix_buf, sizeof(prefix_buf), "%04u", i + 1);
|
|
|
|
uint64_t total = 0;
|
|
|
|
|
|
|
|
Iterator* iter = db_.db->NewIterator(ReadOptions());
|
|
|
|
|
|
|
|
for (iter->Seek(Slice(prefix_buf, 4)); iter->Valid(); iter->Next()) {
|
|
|
|
Slice key = iter->key();
|
|
|
|
|
|
|
|
// stop when we reach a different prefix
|
|
|
|
if (key.ToString().compare(0, 4, prefix_buf) != 0) {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
Slice value = iter->value();
|
|
|
|
uint64_t int_value = std::stoull(value.ToString());
|
|
|
|
if (int_value == 0 || int_value == ULONG_MAX) {
|
|
|
|
fprintf(stderr, "Iter returned unexpected value: %s\n",
|
|
|
|
value.ToString().c_str());
|
|
|
|
abort();
|
|
|
|
}
|
|
|
|
|
|
|
|
total += int_value;
|
|
|
|
}
|
|
|
|
delete iter;
|
|
|
|
|
|
|
|
if (i > 0) {
|
|
|
|
if (total != prev_total) {
|
|
|
|
fprintf(stderr,
|
|
|
|
"RandomTransactionVerify found inconsistent totals. "
|
2015-05-30 18:25:45 +02:00
|
|
|
"Set[%" PRIu32 "]: %" PRIu64 ", Set[%" PRIu32 "]: %" PRIu64
|
|
|
|
" \n",
|
2015-05-29 23:36:35 +02:00
|
|
|
i - 1, prev_total, i, total);
|
|
|
|
abort();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
prev_total = total;
|
|
|
|
}
|
|
|
|
|
2015-08-29 01:09:34 +02:00
|
|
|
fprintf(stdout, "RandomTransactionVerify Success!\n");
|
2015-05-29 23:36:35 +02:00
|
|
|
}
|
2015-10-14 21:43:00 +02:00
|
|
|
#endif // ROCKSDB_LITE
|
2015-05-29 23:36:35 +02:00
|
|
|
|
Support for SingleDelete()
Summary:
This patch fixes #7460559. It introduces SingleDelete as a new database
operation. This operation can be used to delete keys that were never
overwritten (no put following another put of the same key). If an overwritten
key is single deleted the behavior is undefined. Single deletion of a
non-existent key has no effect but multiple consecutive single deletions are
not allowed (see limitations).
In contrast to the conventional Delete() operation, the deletion entry is
removed along with the value when the two are lined up in a compaction. Note:
The semantics are similar to @igor's prototype that allowed to have this
behavior on the granularity of a column family (
https://reviews.facebook.net/D42093 ). This new patch, however, is more
aggressive when it comes to removing tombstones: It removes the SingleDelete
together with the value whenever there is no snapshot between them while the
older patch only did this when the sequence number of the deletion was older
than the earliest snapshot.
Most of the complex additions are in the Compaction Iterator, all other changes
should be relatively straightforward. The patch also includes basic support for
single deletions in db_stress and db_bench.
Limitations:
- Not compatible with cuckoo hash tables
- Single deletions cannot be used in combination with merges and normal
deletions on the same key (other keys are not affected by this)
- Consecutive single deletions are currently not allowed (and older version of
this patch supported this so it could be resurrected if needed)
Test Plan: make all check
Reviewers: yhchiang, sdong, rven, anthony, yoshinorim, igor
Reviewed By: igor
Subscribers: maykov, dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D43179
2015-09-17 20:42:56 +02:00
|
|
|
// Writes and deletes random keys without overwriting keys.
|
|
|
|
//
|
|
|
|
// This benchmark is intended to partially replicate the behavior of MyRocks
|
|
|
|
// secondary indices: All data is stored in keys and updates happen by
|
|
|
|
// deleting the old version of the key and inserting the new version.
|
|
|
|
void RandomReplaceKeys(ThreadState* thread) {
|
|
|
|
std::unique_ptr<const char[]> key_guard;
|
|
|
|
Slice key = AllocateKey(&key_guard);
|
|
|
|
std::vector<uint32_t> counters(FLAGS_numdistinct, 0);
|
|
|
|
size_t max_counter = 50;
|
|
|
|
RandomGenerator gen;
|
|
|
|
|
|
|
|
Status s;
|
|
|
|
DB* db = SelectDB(thread);
|
|
|
|
for (int64_t i = 0; i < FLAGS_numdistinct; i++) {
|
|
|
|
GenerateKeyFromInt(i * max_counter, FLAGS_num, &key);
|
|
|
|
s = db->Put(write_options_, key, gen.Generate(value_size_));
|
|
|
|
if (!s.ok()) {
|
|
|
|
fprintf(stderr, "Operation failed: %s\n", s.ToString().c_str());
|
|
|
|
exit(1);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
db->GetSnapshot();
|
|
|
|
|
|
|
|
std::default_random_engine generator;
|
|
|
|
std::normal_distribution<double> distribution(FLAGS_numdistinct / 2.0,
|
|
|
|
FLAGS_stddev);
|
|
|
|
Duration duration(FLAGS_duration, FLAGS_num);
|
|
|
|
while (!duration.Done(1)) {
|
|
|
|
int64_t rnd_id = static_cast<int64_t>(distribution(generator));
|
|
|
|
int64_t key_id = std::max(std::min(FLAGS_numdistinct - 1, rnd_id),
|
|
|
|
static_cast<int64_t>(0));
|
|
|
|
GenerateKeyFromInt(key_id * max_counter + counters[key_id], FLAGS_num,
|
|
|
|
&key);
|
|
|
|
s = FLAGS_use_single_deletes ? db->SingleDelete(write_options_, key)
|
|
|
|
: db->Delete(write_options_, key);
|
|
|
|
if (s.ok()) {
|
|
|
|
counters[key_id] = (counters[key_id] + 1) % max_counter;
|
|
|
|
GenerateKeyFromInt(key_id * max_counter + counters[key_id], FLAGS_num,
|
|
|
|
&key);
|
|
|
|
s = db->Put(write_options_, key, Slice());
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!s.ok()) {
|
|
|
|
fprintf(stderr, "Operation failed: %s\n", s.ToString().c_str());
|
|
|
|
exit(1);
|
|
|
|
}
|
|
|
|
|
|
|
|
thread->stats.FinishedOps(nullptr, db, 1);
|
|
|
|
}
|
|
|
|
|
|
|
|
char msg[200];
|
|
|
|
snprintf(msg, sizeof(msg),
|
|
|
|
"use single deletes: %d, "
|
|
|
|
"standard deviation: %lf\n",
|
|
|
|
FLAGS_use_single_deletes, FLAGS_stddev);
|
|
|
|
thread->stats.AddMessage(msg);
|
|
|
|
}
|
|
|
|
|
2011-08-22 23:08:51 +02:00
|
|
|
void Compact(ThreadState* thread) {
|
2014-04-11 21:15:09 +02:00
|
|
|
DB* db = SelectDB(thread);
|
2015-06-17 23:36:14 +02:00
|
|
|
db->CompactRange(CompactRangeOptions(), nullptr, nullptr);
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2012-04-17 17:36:46 +02:00
|
|
|
void PrintStats(const char* key) {
|
2014-08-19 03:15:01 +02:00
|
|
|
if (db_.db != nullptr) {
|
|
|
|
PrintStats(db_.db, key, false);
|
2014-04-11 21:15:09 +02:00
|
|
|
}
|
2014-08-19 03:15:01 +02:00
|
|
|
for (const auto& db_with_cfh : multi_dbs_) {
|
|
|
|
PrintStats(db_with_cfh.db, key, true);
|
2014-04-11 21:15:09 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void PrintStats(DB* db, const char* key, bool print_header = false) {
|
|
|
|
if (print_header) {
|
|
|
|
fprintf(stdout, "\n==== DB: %s ===\n", db->GetName().c_str());
|
|
|
|
}
|
2011-04-12 21:38:58 +02:00
|
|
|
std::string stats;
|
2014-04-11 21:15:09 +02:00
|
|
|
if (!db->GetProperty(key, &stats)) {
|
2011-08-22 23:08:51 +02:00
|
|
|
stats = "(failed)";
|
2011-04-12 21:38:58 +02:00
|
|
|
}
|
2011-08-22 23:08:51 +02:00
|
|
|
fprintf(stdout, "\n%s\n", stats.c_str());
|
2011-04-12 21:38:58 +02:00
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
};
|
|
|
|
|
2013-10-04 06:49:15 +02:00
|
|
|
} // namespace rocksdb
|
2011-03-18 23:37:00 +01:00
|
|
|
|
|
|
|
int main(int argc, char** argv) {
|
2014-04-23 15:11:35 +02:00
|
|
|
rocksdb::port::InstallStackTraceHandler();
|
2014-05-09 02:25:13 +02:00
|
|
|
SetUsageMessage(std::string("\nUSAGE:\n") + std::string(argv[0]) +
|
|
|
|
" [OPTIONS]...");
|
|
|
|
ParseCommandLineFlags(&argc, &argv, true);
|
2013-04-11 19:54:35 +02:00
|
|
|
|
2013-10-24 16:43:14 +02:00
|
|
|
FLAGS_compaction_style_e = (rocksdb::CompactionStyle) FLAGS_compaction_style;
|
|
|
|
if (FLAGS_statistics) {
|
|
|
|
dbstats = rocksdb::CreateDBStatistics();
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
2015-09-22 02:16:31 +02:00
|
|
|
FLAGS_compaction_pri_e = (rocksdb::CompactionPri)FLAGS_compaction_pri;
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2014-11-21 17:05:28 +01:00
|
|
|
std::vector<std::string> fanout = rocksdb::StringSplit(
|
|
|
|
FLAGS_max_bytes_for_level_multiplier_additional, ',');
|
2013-10-24 16:43:14 +02:00
|
|
|
for (unsigned int j= 0; j < fanout.size(); j++) {
|
|
|
|
FLAGS_max_bytes_for_level_multiplier_additional_v.push_back(
|
2015-06-08 20:43:55 +02:00
|
|
|
#ifndef CYGWIN
|
|
|
|
std::stoi(fanout[j]));
|
|
|
|
#else
|
|
|
|
stoi(fanout[j]));
|
|
|
|
#endif
|
2013-10-24 16:43:14 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
FLAGS_compression_type_e =
|
|
|
|
StringToCompressionType(FLAGS_compression_type.c_str());
|
|
|
|
|
|
|
|
if (!FLAGS_hdfs.empty()) {
|
|
|
|
FLAGS_env = new rocksdb::HdfsEnv(FLAGS_hdfs);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!strcasecmp(FLAGS_compaction_fadvice.c_str(), "NONE"))
|
|
|
|
FLAGS_compaction_fadvice_e = rocksdb::Options::NONE;
|
|
|
|
else if (!strcasecmp(FLAGS_compaction_fadvice.c_str(), "NORMAL"))
|
|
|
|
FLAGS_compaction_fadvice_e = rocksdb::Options::NORMAL;
|
|
|
|
else if (!strcasecmp(FLAGS_compaction_fadvice.c_str(), "SEQUENTIAL"))
|
|
|
|
FLAGS_compaction_fadvice_e = rocksdb::Options::SEQUENTIAL;
|
|
|
|
else if (!strcasecmp(FLAGS_compaction_fadvice.c_str(), "WILLNEED"))
|
|
|
|
FLAGS_compaction_fadvice_e = rocksdb::Options::WILLNEED;
|
|
|
|
else {
|
|
|
|
fprintf(stdout, "Unknown compaction fadvice:%s\n",
|
|
|
|
FLAGS_compaction_fadvice.c_str());
|
|
|
|
}
|
|
|
|
|
|
|
|
FLAGS_rep_factory = StringToRepFactory(FLAGS_memtablerep.c_str());
|
|
|
|
|
2012-10-19 23:00:53 +02:00
|
|
|
// The number of background threads should be at least as much the
|
|
|
|
// max number of concurrent compactions.
|
|
|
|
FLAGS_env->SetBackgroundThreads(FLAGS_max_background_compactions);
|
2014-10-10 05:07:12 +02:00
|
|
|
FLAGS_env->SetBackgroundThreads(FLAGS_max_background_flushes,
|
|
|
|
rocksdb::Env::Priority::HIGH);
|
|
|
|
|
2012-08-27 08:45:35 +02:00
|
|
|
// Choose a location for the test database if none given with --db=<path>
|
2013-10-24 16:43:14 +02:00
|
|
|
if (FLAGS_db.empty()) {
|
|
|
|
std::string default_db_path;
|
|
|
|
rocksdb::Env::Default()->GetTestDirectory(&default_db_path);
|
|
|
|
default_db_path += "/dbbench";
|
|
|
|
FLAGS_db = default_db_path;
|
2012-08-27 08:45:35 +02:00
|
|
|
}
|
|
|
|
|
2015-03-30 21:58:32 +02:00
|
|
|
if (FLAGS_stats_interval_seconds > 0) {
|
|
|
|
// When both are set then FLAGS_stats_interval determines the frequency
|
|
|
|
// at which the timer is checked for FLAGS_stats_interval_seconds
|
|
|
|
FLAGS_stats_interval = 1000;
|
|
|
|
}
|
|
|
|
|
2013-10-04 06:49:15 +02:00
|
|
|
rocksdb::Benchmark benchmark;
|
2011-03-18 23:37:00 +01:00
|
|
|
benchmark.Run();
|
|
|
|
return 0;
|
|
|
|
}
|
2014-05-09 02:25:13 +02:00
|
|
|
|
|
|
|
#endif // GFLAGS
|