2016-02-10 00:12:00 +01:00
|
|
|
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
2017-07-16 01:03:42 +02:00
|
|
|
// This source code is licensed under both the GPLv2 (found in the
|
|
|
|
// COPYING file in the root directory) and Apache 2.0 License
|
|
|
|
// (found in the LICENSE.Apache file in the root directory).
|
2013-10-16 23:59:46 +02:00
|
|
|
//
|
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.
|
|
|
|
|
|
|
|
#include "port/port_posix.h"
|
|
|
|
|
2014-03-26 19:24:52 +01:00
|
|
|
#include <assert.h>
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
#if defined(__i386__) || defined(__x86_64__)
|
|
|
|
#include <cpuid.h>
|
|
|
|
#endif
|
2014-07-03 19:22:08 +02:00
|
|
|
#include <errno.h>
|
2017-05-10 20:50:10 +02:00
|
|
|
#include <sched.h>
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
#include <signal.h>
|
|
|
|
#include <stdio.h>
|
2011-03-18 23:37:00 +01:00
|
|
|
#include <string.h>
|
2015-09-10 19:49:28 +02:00
|
|
|
#include <sys/resource.h>
|
2019-06-01 02:19:43 +02:00
|
|
|
#include <sys/time.h>
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
#include <unistd.h>
|
2014-07-03 19:22:08 +02:00
|
|
|
#include <cstdlib>
|
2019-06-01 02:19:43 +02:00
|
|
|
#include "logging/logging.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2020-02-20 21:07:53 +01:00
|
|
|
namespace ROCKSDB_NAMESPACE {
|
2019-03-20 20:24:57 +01:00
|
|
|
|
|
|
|
// We want to give users opportunity to default all the mutexes to adaptive if
|
|
|
|
// not specified otherwise. This enables a quick way to conduct various
|
|
|
|
// performance related experiements.
|
|
|
|
//
|
|
|
|
// NB! Support for adaptive mutexes is turned on by definining
|
|
|
|
// ROCKSDB_PTHREAD_ADAPTIVE_MUTEX during the compilation. If you use RocksDB
|
|
|
|
// build environment then this happens automatically; otherwise it's up to the
|
|
|
|
// consumer to define the identifier.
|
|
|
|
#ifdef ROCKSDB_DEFAULT_TO_ADAPTIVE_MUTEX
|
|
|
|
extern const bool kDefaultToAdaptiveMutex = true;
|
|
|
|
#else
|
|
|
|
extern const bool kDefaultToAdaptiveMutex = false;
|
|
|
|
#endif
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
namespace port {
|
|
|
|
|
2014-07-04 00:47:02 +02:00
|
|
|
static int PthreadCall(const char* label, int result) {
|
|
|
|
if (result != 0 && result != ETIMEDOUT) {
|
2011-03-18 23:37:00 +01:00
|
|
|
fprintf(stderr, "pthread %s: %s\n", label, strerror(result));
|
|
|
|
abort();
|
|
|
|
}
|
2014-07-04 00:47:02 +02:00
|
|
|
return result;
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2018-02-22 21:36:41 +01:00
|
|
|
Mutex::Mutex(bool adaptive) {
|
2018-03-05 22:08:17 +01:00
|
|
|
(void) adaptive;
|
2016-04-23 01:49:12 +02:00
|
|
|
#ifdef ROCKSDB_PTHREAD_ADAPTIVE_MUTEX
|
2013-06-01 01:30:17 +02:00
|
|
|
if (!adaptive) {
|
generic rate limiter
Summary:
A generic rate limiter that can be shared by threads and rocksdb
instances. Will use this to smooth out write traffic generated by
compaction and flush. This will help us get better p99 behavior on flash
storage.
Test Plan:
unit test output
==== Test RateLimiterTest.Rate
request size [1 - 1023], limit 10 KB/sec, actual rate: 10.374969 KB/sec, elapsed 2002265
request size [1 - 2047], limit 20 KB/sec, actual rate: 20.771242 KB/sec, elapsed 2002139
request size [1 - 4095], limit 40 KB/sec, actual rate: 41.285299 KB/sec, elapsed 2202424
request size [1 - 8191], limit 80 KB/sec, actual rate: 81.371605 KB/sec, elapsed 2402558
request size [1 - 16383], limit 160 KB/sec, actual rate: 162.541268 KB/sec, elapsed 3303500
Reviewers: yhchiang, igor, sdong
Reviewed By: sdong
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D19359
2014-07-08 20:41:57 +02:00
|
|
|
PthreadCall("init mutex", pthread_mutex_init(&mu_, nullptr));
|
2013-06-01 01:30:17 +02:00
|
|
|
} else {
|
|
|
|
pthread_mutexattr_t mutex_attr;
|
|
|
|
PthreadCall("init mutex attr", pthread_mutexattr_init(&mutex_attr));
|
|
|
|
PthreadCall("set mutex attr",
|
|
|
|
pthread_mutexattr_settype(&mutex_attr,
|
|
|
|
PTHREAD_MUTEX_ADAPTIVE_NP));
|
|
|
|
PthreadCall("init mutex", pthread_mutex_init(&mu_, &mutex_attr));
|
|
|
|
PthreadCall("destroy mutex attr",
|
|
|
|
pthread_mutexattr_destroy(&mutex_attr));
|
|
|
|
}
|
2016-04-23 01:49:12 +02:00
|
|
|
#else
|
generic rate limiter
Summary:
A generic rate limiter that can be shared by threads and rocksdb
instances. Will use this to smooth out write traffic generated by
compaction and flush. This will help us get better p99 behavior on flash
storage.
Test Plan:
unit test output
==== Test RateLimiterTest.Rate
request size [1 - 1023], limit 10 KB/sec, actual rate: 10.374969 KB/sec, elapsed 2002265
request size [1 - 2047], limit 20 KB/sec, actual rate: 20.771242 KB/sec, elapsed 2002139
request size [1 - 4095], limit 40 KB/sec, actual rate: 41.285299 KB/sec, elapsed 2202424
request size [1 - 8191], limit 80 KB/sec, actual rate: 81.371605 KB/sec, elapsed 2402558
request size [1 - 16383], limit 160 KB/sec, actual rate: 162.541268 KB/sec, elapsed 3303500
Reviewers: yhchiang, igor, sdong
Reviewed By: sdong
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D19359
2014-07-08 20:41:57 +02:00
|
|
|
PthreadCall("init mutex", pthread_mutex_init(&mu_, nullptr));
|
2016-04-23 01:49:12 +02:00
|
|
|
#endif // ROCKSDB_PTHREAD_ADAPTIVE_MUTEX
|
2013-06-01 01:30:17 +02:00
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
|
|
|
|
Mutex::~Mutex() { PthreadCall("destroy mutex", pthread_mutex_destroy(&mu_)); }
|
|
|
|
|
2014-03-26 19:24:52 +01:00
|
|
|
void Mutex::Lock() {
|
|
|
|
PthreadCall("lock", pthread_mutex_lock(&mu_));
|
|
|
|
#ifndef NDEBUG
|
2014-04-23 03:38:10 +02:00
|
|
|
locked_ = true;
|
2014-03-26 19:24:52 +01:00
|
|
|
#endif
|
|
|
|
}
|
|
|
|
|
|
|
|
void Mutex::Unlock() {
|
|
|
|
#ifndef NDEBUG
|
2014-04-23 03:38:10 +02:00
|
|
|
locked_ = false;
|
2014-03-26 19:24:52 +01:00
|
|
|
#endif
|
|
|
|
PthreadCall("unlock", pthread_mutex_unlock(&mu_));
|
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2014-03-26 19:24:52 +01:00
|
|
|
void Mutex::AssertHeld() {
|
|
|
|
#ifndef NDEBUG
|
2014-04-23 03:38:10 +02:00
|
|
|
assert(locked_);
|
2014-03-26 19:24:52 +01:00
|
|
|
#endif
|
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
|
|
|
|
CondVar::CondVar(Mutex* mu)
|
|
|
|
: mu_(mu) {
|
generic rate limiter
Summary:
A generic rate limiter that can be shared by threads and rocksdb
instances. Will use this to smooth out write traffic generated by
compaction and flush. This will help us get better p99 behavior on flash
storage.
Test Plan:
unit test output
==== Test RateLimiterTest.Rate
request size [1 - 1023], limit 10 KB/sec, actual rate: 10.374969 KB/sec, elapsed 2002265
request size [1 - 2047], limit 20 KB/sec, actual rate: 20.771242 KB/sec, elapsed 2002139
request size [1 - 4095], limit 40 KB/sec, actual rate: 41.285299 KB/sec, elapsed 2202424
request size [1 - 8191], limit 80 KB/sec, actual rate: 81.371605 KB/sec, elapsed 2402558
request size [1 - 16383], limit 160 KB/sec, actual rate: 162.541268 KB/sec, elapsed 3303500
Reviewers: yhchiang, igor, sdong
Reviewed By: sdong
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D19359
2014-07-08 20:41:57 +02:00
|
|
|
PthreadCall("init cv", pthread_cond_init(&cv_, nullptr));
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
CondVar::~CondVar() { PthreadCall("destroy cv", pthread_cond_destroy(&cv_)); }
|
|
|
|
|
|
|
|
void CondVar::Wait() {
|
2014-03-26 19:24:52 +01:00
|
|
|
#ifndef NDEBUG
|
2014-04-23 03:38:10 +02:00
|
|
|
mu_->locked_ = false;
|
2014-03-26 19:24:52 +01:00
|
|
|
#endif
|
2011-03-18 23:37:00 +01:00
|
|
|
PthreadCall("wait", pthread_cond_wait(&cv_, &mu_->mu_));
|
2014-03-26 19:24:52 +01:00
|
|
|
#ifndef NDEBUG
|
2014-04-23 03:38:10 +02:00
|
|
|
mu_->locked_ = true;
|
2014-03-26 19:24:52 +01:00
|
|
|
#endif
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2014-07-03 19:22:08 +02:00
|
|
|
bool CondVar::TimedWait(uint64_t abs_time_us) {
|
|
|
|
struct timespec ts;
|
2014-11-13 20:39:30 +01:00
|
|
|
ts.tv_sec = static_cast<time_t>(abs_time_us / 1000000);
|
|
|
|
ts.tv_nsec = static_cast<suseconds_t>((abs_time_us % 1000000) * 1000);
|
2014-07-03 19:22:08 +02:00
|
|
|
|
|
|
|
#ifndef NDEBUG
|
|
|
|
mu_->locked_ = false;
|
|
|
|
#endif
|
|
|
|
int err = pthread_cond_timedwait(&cv_, &mu_->mu_, &ts);
|
|
|
|
#ifndef NDEBUG
|
|
|
|
mu_->locked_ = true;
|
|
|
|
#endif
|
|
|
|
if (err == ETIMEDOUT) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
if (err != 0) {
|
|
|
|
PthreadCall("timedwait", err);
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
void CondVar::Signal() {
|
|
|
|
PthreadCall("signal", pthread_cond_signal(&cv_));
|
|
|
|
}
|
|
|
|
|
|
|
|
void CondVar::SignalAll() {
|
|
|
|
PthreadCall("broadcast", pthread_cond_broadcast(&cv_));
|
|
|
|
}
|
|
|
|
|
generic rate limiter
Summary:
A generic rate limiter that can be shared by threads and rocksdb
instances. Will use this to smooth out write traffic generated by
compaction and flush. This will help us get better p99 behavior on flash
storage.
Test Plan:
unit test output
==== Test RateLimiterTest.Rate
request size [1 - 1023], limit 10 KB/sec, actual rate: 10.374969 KB/sec, elapsed 2002265
request size [1 - 2047], limit 20 KB/sec, actual rate: 20.771242 KB/sec, elapsed 2002139
request size [1 - 4095], limit 40 KB/sec, actual rate: 41.285299 KB/sec, elapsed 2202424
request size [1 - 8191], limit 80 KB/sec, actual rate: 81.371605 KB/sec, elapsed 2402558
request size [1 - 16383], limit 160 KB/sec, actual rate: 162.541268 KB/sec, elapsed 3303500
Reviewers: yhchiang, igor, sdong
Reviewed By: sdong
Subscribers: leveldb
Differential Revision: https://reviews.facebook.net/D19359
2014-07-08 20:41:57 +02:00
|
|
|
RWMutex::RWMutex() {
|
|
|
|
PthreadCall("init mutex", pthread_rwlock_init(&mu_, nullptr));
|
|
|
|
}
|
2012-10-02 06:58:36 +02:00
|
|
|
|
|
|
|
RWMutex::~RWMutex() { PthreadCall("destroy mutex", pthread_rwlock_destroy(&mu_)); }
|
|
|
|
|
|
|
|
void RWMutex::ReadLock() { PthreadCall("read lock", pthread_rwlock_rdlock(&mu_)); }
|
|
|
|
|
|
|
|
void RWMutex::WriteLock() { PthreadCall("write lock", pthread_rwlock_wrlock(&mu_)); }
|
|
|
|
|
2014-06-17 00:41:46 +02:00
|
|
|
void RWMutex::ReadUnlock() { PthreadCall("read unlock", pthread_rwlock_unlock(&mu_)); }
|
|
|
|
|
|
|
|
void RWMutex::WriteUnlock() { PthreadCall("write unlock", pthread_rwlock_unlock(&mu_)); }
|
2012-10-02 06:58:36 +02:00
|
|
|
|
2017-05-10 20:50:10 +02:00
|
|
|
int PhysicalCoreID() {
|
|
|
|
#if defined(ROCKSDB_SCHED_GETCPU_PRESENT) && defined(__x86_64__) && \
|
|
|
|
(__GNUC__ > 2 || (__GNUC__ == 2 && __GNUC_MINOR__ >= 22))
|
|
|
|
// sched_getcpu uses VDSO getcpu() syscall since 2.22. I believe Linux offers VDSO
|
|
|
|
// support only on x86_64. This is the fastest/preferred method if available.
|
2017-05-10 03:43:23 +02:00
|
|
|
int cpuno = sched_getcpu();
|
|
|
|
if (cpuno < 0) {
|
|
|
|
return -1;
|
|
|
|
}
|
2017-05-10 20:50:10 +02:00
|
|
|
return cpuno;
|
|
|
|
#elif defined(__x86_64__) || defined(__i386__)
|
|
|
|
// clang/gcc both provide cpuid.h, which defines __get_cpuid(), for x86_64 and i386.
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
unsigned eax, ebx = 0, ecx, edx;
|
2017-05-03 21:48:23 +02:00
|
|
|
if (!__get_cpuid(1, &eax, &ebx, &ecx, &edx)) {
|
|
|
|
return -1;
|
|
|
|
}
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
return ebx >> 24;
|
|
|
|
#else
|
2017-05-10 20:50:10 +02:00
|
|
|
// give up, the caller can generate a random number or something.
|
|
|
|
return -1;
|
support for concurrent adds to memtable
Summary:
This diff adds support for concurrent adds to the skiplist memtable
implementations. Memory allocation is made thread-safe by the addition of
a spinlock, with small per-core buffers to avoid contention. Concurrent
memtable writes are made via an additional method and don't impose a
performance overhead on the non-concurrent case, so parallelism can be
selected on a per-batch basis.
Write thread synchronization is an increasing bottleneck for higher levels
of concurrency, so this diff adds --enable_write_thread_adaptive_yield
(default off). This feature causes threads joining a write batch
group to spin for a short time (default 100 usec) using sched_yield,
rather than going to sleep on a mutex. If the timing of the yield calls
indicates that another thread has actually run during the yield then
spinning is avoided. This option improves performance for concurrent
situations even without parallel adds, although it has the potential to
increase CPU usage (and the heuristic adaptation is not yet mature).
Parallel writes are not currently compatible with
inplace updates, update callbacks, or delete filtering.
Enable it with --allow_concurrent_memtable_write (and
--enable_write_thread_adaptive_yield). Parallel memtable writes
are performance neutral when there is no actual parallelism, and in
my experiments (SSD server-class Linux and varying contention and key
sizes for fillrandom) they are always a performance win when there is
more than one thread.
Statistics are updated earlier in the write path, dropping the number
of DB mutex acquisitions from 2 to 1 for almost all cases.
This diff was motivated and inspired by Yahoo's cLSM work. It is more
conservative than cLSM: RocksDB's write batch group leader role is
preserved (along with all of the existing flush and write throttling
logic) and concurrent writers are blocked until all memtable insertions
have completed and the sequence number has been advanced, to preserve
linearizability.
My test config is "db_bench -benchmarks=fillrandom -threads=$T
-batch_size=1 -memtablerep=skip_list -value_size=100 --num=1000000/$T
-level0_slowdown_writes_trigger=9999 -level0_stop_writes_trigger=9999
-disable_auto_compactions --max_write_buffer_number=8
-max_background_flushes=8 --disable_wal --write_buffer_size=160000000
--block_size=16384 --allow_concurrent_memtable_write" on a two-socket
Xeon E5-2660 @ 2.2Ghz with lots of memory and an SSD hard drive. With 1
thread I get ~440Kops/sec. Peak performance for 1 socket (numactl
-N1) is slightly more than 1Mops/sec, at 16 threads. Peak performance
across both sockets happens at 30 threads, and is ~900Kops/sec, although
with fewer threads there is less performance loss when the system has
background work.
Test Plan:
1. concurrent stress tests for InlineSkipList and DynamicBloom
2. make clean; make check
3. make clean; DISABLE_JEMALLOC=1 make valgrind_check; valgrind db_bench
4. make clean; COMPILE_WITH_TSAN=1 make all check; db_bench
5. make clean; COMPILE_WITH_ASAN=1 make all check; db_bench
6. make clean; OPT=-DROCKSDB_LITE make check
7. verify no perf regressions when disabled
Reviewers: igor, sdong
Reviewed By: sdong
Subscribers: MarkCallaghan, IslamAbdelRahman, anthony, yhchiang, rven, sdong, guyg8, kradhakrishnan, dhruba
Differential Revision: https://reviews.facebook.net/D50589
2015-08-15 01:59:07 +02:00
|
|
|
#endif
|
|
|
|
}
|
|
|
|
|
2012-08-27 08:45:35 +02:00
|
|
|
void InitOnce(OnceType* once, void (*initializer)()) {
|
|
|
|
PthreadCall("once", pthread_once(once, initializer));
|
|
|
|
}
|
|
|
|
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
void Crash(const std::string& srcfile, int srcline) {
|
|
|
|
fprintf(stdout, "Crashing at %s:%d\n", srcfile.c_str(), srcline);
|
|
|
|
fflush(stdout);
|
|
|
|
kill(getpid(), SIGTERM);
|
|
|
|
}
|
|
|
|
|
2015-09-10 19:49:28 +02:00
|
|
|
int GetMaxOpenFiles() {
|
|
|
|
#if defined(RLIMIT_NOFILE)
|
|
|
|
struct rlimit no_files_limit;
|
|
|
|
if (getrlimit(RLIMIT_NOFILE, &no_files_limit) != 0) {
|
|
|
|
return -1;
|
|
|
|
}
|
|
|
|
// protect against overflow
|
2019-07-30 23:09:02 +02:00
|
|
|
if (static_cast<uintmax_t>(no_files_limit.rlim_cur) >=
|
|
|
|
static_cast<uintmax_t>(std::numeric_limits<int>::max())) {
|
2015-09-10 19:49:28 +02:00
|
|
|
return std::numeric_limits<int>::max();
|
|
|
|
}
|
|
|
|
return static_cast<int>(no_files_limit.rlim_cur);
|
|
|
|
#endif
|
|
|
|
return -1;
|
|
|
|
}
|
|
|
|
|
2017-07-24 19:46:21 +02:00
|
|
|
void *cacheline_aligned_alloc(size_t size) {
|
2017-08-30 06:41:46 +02:00
|
|
|
#if __GNUC__ < 5 && defined(__SANITIZE_ADDRESS__)
|
|
|
|
return malloc(size);
|
2017-07-24 19:46:21 +02:00
|
|
|
#elif ( _POSIX_C_SOURCE >= 200112L || _XOPEN_SOURCE >= 600 || defined(__APPLE__))
|
|
|
|
void *m;
|
|
|
|
errno = posix_memalign(&m, CACHE_LINE_SIZE, size);
|
2018-03-07 21:39:19 +01:00
|
|
|
return errno ? nullptr : m;
|
2017-07-24 19:46:21 +02:00
|
|
|
#else
|
|
|
|
return malloc(size);
|
|
|
|
#endif
|
|
|
|
}
|
|
|
|
|
|
|
|
void cacheline_aligned_free(void *memblock) {
|
|
|
|
free(memblock);
|
|
|
|
}
|
|
|
|
|
2020-02-22 16:59:38 +01:00
|
|
|
static size_t GetPageSize() {
|
|
|
|
#if defined(OS_LINUX) || defined(_SC_PAGESIZE)
|
|
|
|
long v = sysconf(_SC_PAGESIZE);
|
|
|
|
if (v >= 1024) {
|
|
|
|
return static_cast<size_t>(v);
|
|
|
|
}
|
|
|
|
#endif
|
|
|
|
// Default assume 4KB
|
|
|
|
return 4U * 1024U;
|
|
|
|
}
|
|
|
|
|
|
|
|
const size_t kPageSize = GetPageSize();
|
2017-07-24 19:46:21 +02:00
|
|
|
|
2011-10-31 18:22:06 +01:00
|
|
|
} // namespace port
|
2020-02-20 21:07:53 +01:00
|
|
|
} // namespace ROCKSDB_NAMESPACE
|