mirror of
https://github.com/facebook/rocksdb.git
synced 2024-11-26 07:30:54 +00:00
c70bae1b05
Summary: Previously the shutdown process did not properly wait for all `compaction_thread_limiter` tokens to be released before proceeding to delete the DB's C++ objects. When this happened, we saw tests like "DBCompactionTest.CompactionLimiter" flake with the following error: ``` virtual rocksdb::ConcurrentTaskLimiterImpl::~ConcurrentTaskLimiterImpl(): Assertion `outstanding_tasks_ == 0' failed. ``` There is a case where a token can still be alive even after the shutdown process has waited for BG work to complete. In particular, this happens because the shutdown process only waits for flush/compaction scheduled/unscheduled counters to all reach zero. These counters are decremented in `BackgroundCallCompaction()` functions. However, tokens are released in `BGWork*Compaction()` functions, which actually wrap the `BackgroundCallCompaction()` function. A simple sleep could repro the race condition: ``` $ diff --git a/db/db_impl/db_impl_compaction_flush.cc b/db/db_impl/db_impl_compaction_flush.cc index 806bc548a..ba59efa89 100644 --- a/db/db_impl/db_impl_compaction_flush.cc +++ b/db/db_impl/db_impl_compaction_flush.cc @@ -2442,6 +2442,7 @@ void DBImpl::BGWorkCompaction(void* arg) { static_cast<PrepickedCompaction*>(ca.prepicked_compaction); static_cast_with_check<DBImpl>(ca.db)->BackgroundCallCompaction( prepicked_compaction, Env::Priority::LOW); + sleep(1); delete prepicked_compaction; } $ ./db_compaction_test --gtest_filter=DBCompactionTest.CompactionLimiter db_compaction_test: util/concurrent_task_limiter_impl.cc:24: virtual rocksdb::ConcurrentTaskLimiterImpl::~ConcurrentTaskLimiterImpl(): Assertion `outstanding_tasks_ == 0' failed. Received signal 6 (Aborted) #0 /usr/local/fbcode/platform007/lib/libc.so.6(gsignal+0xcf) [0x7f02673c30ff] ?? ??:0 https://github.com/facebook/rocksdb/issues/1 /usr/local/fbcode/platform007/lib/libc.so.6(abort+0x134) [0x7f02673ac934] ?? ??:0 ... ``` Pull Request resolved: https://github.com/facebook/rocksdb/pull/8253 Test Plan: sleeps to expose race conditions Reviewed By: akankshamahajan15 Differential Revision: D28168064 Pulled By: ajkr fbshipit-source-id: 9e5167c74398d323e7975980c5cc00f450631160
73 lines
2.3 KiB
C++
73 lines
2.3 KiB
C++
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
|
// 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).
|
|
//
|
|
// 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 "util/concurrent_task_limiter_impl.h"
|
|
#include "rocksdb/concurrent_task_limiter.h"
|
|
|
|
namespace ROCKSDB_NAMESPACE {
|
|
|
|
ConcurrentTaskLimiterImpl::ConcurrentTaskLimiterImpl(
|
|
const std::string& name, int32_t max_outstanding_task)
|
|
: name_(name),
|
|
max_outstanding_tasks_{max_outstanding_task},
|
|
outstanding_tasks_{0} {
|
|
|
|
}
|
|
|
|
ConcurrentTaskLimiterImpl::~ConcurrentTaskLimiterImpl() {
|
|
assert(outstanding_tasks_ == 0);
|
|
}
|
|
|
|
const std::string& ConcurrentTaskLimiterImpl::GetName() const {
|
|
return name_;
|
|
}
|
|
|
|
void ConcurrentTaskLimiterImpl::SetMaxOutstandingTask(int32_t limit) {
|
|
max_outstanding_tasks_.store(limit, std::memory_order_relaxed);
|
|
}
|
|
|
|
void ConcurrentTaskLimiterImpl::ResetMaxOutstandingTask() {
|
|
max_outstanding_tasks_.store(-1, std::memory_order_relaxed);
|
|
}
|
|
|
|
int32_t ConcurrentTaskLimiterImpl::GetOutstandingTask() const {
|
|
return outstanding_tasks_.load(std::memory_order_relaxed);
|
|
}
|
|
|
|
std::unique_ptr<TaskLimiterToken> ConcurrentTaskLimiterImpl::GetToken(
|
|
bool force) {
|
|
int32_t limit = max_outstanding_tasks_.load(std::memory_order_relaxed);
|
|
int32_t tasks = outstanding_tasks_.load(std::memory_order_relaxed);
|
|
// force = true, bypass the throttle.
|
|
// limit < 0 means unlimited tasks.
|
|
while (force || limit < 0 || tasks < limit) {
|
|
if (outstanding_tasks_.compare_exchange_weak(tasks, tasks + 1)) {
|
|
return std::unique_ptr<TaskLimiterToken>(new TaskLimiterToken(this));
|
|
}
|
|
}
|
|
return nullptr;
|
|
}
|
|
|
|
ConcurrentTaskLimiter* NewConcurrentTaskLimiter(
|
|
const std::string& name, int32_t limit) {
|
|
return new ConcurrentTaskLimiterImpl(name, limit);
|
|
}
|
|
|
|
void TaskLimiterToken::ReleaseOnce() {
|
|
if (!released_) {
|
|
--limiter_->outstanding_tasks_;
|
|
released_ = true;
|
|
}
|
|
assert(limiter_->outstanding_tasks_ >= 0);
|
|
}
|
|
|
|
TaskLimiterToken::~TaskLimiterToken() { ReleaseOnce(); }
|
|
|
|
} // namespace ROCKSDB_NAMESPACE
|