Merge pull request #72969 from Algunenano/shared_mutex_other_os

Use absl::Mutex instead of std::shared_mutex on non-Linux OSs
This commit is contained in:
Raúl Marín 2024-12-11 12:12:05 +00:00 committed by GitHub
commit ca2813bacf
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
15 changed files with 49 additions and 22 deletions

View File

@ -112,12 +112,7 @@ private:
/// Protects saved values.
mutable SharedMutex values_mutex;
/// Values store the result of the last update prepared for reading.
#ifdef OS_LINUX
AsynchronousMetricValues values TSA_GUARDED_BY(values_mutex);
#else
/// When SharedMutex == std::shared_mutex it may not be annotated with the 'capability'.
AsynchronousMetricValues values;
#endif
#if defined(OS_LINUX) || defined(OS_FREEBSD)
MemoryStatisticsOS memory_stat TSA_GUARDED_BY(data_mutex);

View File

@ -1,11 +1,12 @@
#include "OvercommitTracker.h"
#include <chrono>
#include <mutex>
#include <Common/ProfileEvents.h>
#include <Common/CurrentMetrics.h>
#include <Interpreters/ProcessList.h>
#include <chrono>
#include <shared_mutex>
namespace CurrentMetrics
{
extern const Metric ThreadsInOvercommitTracker;

View File

@ -4,6 +4,8 @@
#include <Common/ProfileEvents.h>
#include <Common/Stopwatch.h>
#include <shared_mutex>
namespace DB
{

View File

@ -1,8 +1,8 @@
#ifdef OS_LINUX /// Because of futex
#include <Common/SharedMutex.h>
#include <base/getThreadId.h>
#ifdef OS_LINUX /// Because of futex
#include <bit>
#include <Common/futex.h>

View File

@ -1,11 +1,11 @@
#pragma once
#include <shared_mutex>
#include <base/defines.h>
#ifdef OS_LINUX /// Because of futex
#include <base/types.h>
#include <base/defines.h>
#include <atomic>
namespace DB
@ -46,11 +46,35 @@ private:
#else
#include <absl/synchronization/mutex.h>
namespace DB
{
using SharedMutex = std::shared_mutex;
class TSA_CAPABILITY("SharedMutex") SharedMutex : public absl::Mutex
{
using absl::Mutex::Mutex;
public:
SharedMutex(const SharedMutex &) = delete;
SharedMutex & operator=(const SharedMutex &) = delete;
SharedMutex(SharedMutex &&) = delete;
SharedMutex & operator=(SharedMutex &&) = delete;
// Exclusive ownership
void lock() TSA_ACQUIRE() { WriterLock(); }
bool try_lock() TSA_TRY_ACQUIRE(true) { return WriterTryLock(); }
void unlock() TSA_RELEASE() { WriterUnlock(); }
// Shared ownership
void lock_shared() TSA_ACQUIRE_SHARED() { ReaderLock(); }
bool try_lock_shared() TSA_TRY_ACQUIRE_SHARED(true) { return ReaderTryLock(); }
void unlock_shared() TSA_RELEASE_SHARED() { ReaderUnlock(); }
};
}
#endif

View File

@ -19,9 +19,10 @@
#include <filesystem>
#include <map>
#include <mutex>
#include <shared_mutex>
#include <unordered_map>
#include <fmt/format.h>
#include <libunwind.h>
#include <fmt/format.h>
#include <boost/algorithm/string/split.hpp>

View File

@ -1,6 +1,7 @@
#include <atomic>
#include <cerrno>
#include <chrono>
#include <shared_mutex>
#include <Coordination/CoordinationSettings.h>
#include <Coordination/KeeperDispatcher.h>
#include <Coordination/KeeperReconfiguration.h>

View File

@ -30,6 +30,7 @@
#include <Coordination/KeeperStorage.h>
#include <functional>
#include <shared_mutex>
#include <base/defines.h>
namespace ProfileEvents

View File

@ -65,14 +65,8 @@ struct InMemoryDirectoryPathMap
mutable SharedMutex mutex;
#ifdef OS_LINUX
FileNames TSA_GUARDED_BY(mutex) unique_filenames;
Map TSA_GUARDED_BY(mutex) map;
/// std::shared_mutex may not be annotated with the 'capability' attribute in libcxx.
#else
FileNames unique_filenames;
Map map;
#endif
};
}

View File

@ -1,10 +1,9 @@
#include <Disks/ObjectStorages/MetadataStorageFromDisk.h>
#include <Disks/ObjectStorages/IMetadataStorage.h>
#include <Common/getRandomASCIIString.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <ranges>
#include <filesystem>
#include <shared_mutex>
namespace DB

View File

@ -9,6 +9,8 @@
#include <Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h>
#include <Disks/ObjectStorages/MetadataStorageTransactionState.h>
#include <shared_mutex>
namespace DB
{

View File

@ -18,6 +18,7 @@
#include <memory>
#include <mutex>
#include <set>
#include <shared_mutex>
#include <unordered_map>
#include <unordered_set>

View File

@ -1,4 +1,5 @@
#include <optional>
#include <shared_mutex>
#include <Core/Field.h>

View File

@ -1,7 +1,10 @@
#include <Processors/Executors/ExecutingGraph.h>
#include <stack>
#include <Common/Stopwatch.h>
#include <shared_mutex>
#include <stack>
namespace DB
{

View File

@ -15,6 +15,8 @@
#include <Poco/Net/HTTPBasicCredentials.h>
#include <Poco/Util/LayeredConfiguration.h>
#include <shared_mutex>
namespace DB
{