liblog: use a rwlock for writer initialization
The current system of using atomics isn't thread safe and may result
in doubly closing FDs or closing actively used FDs. The safest way to
do this is to use a rwlock, which should not have a much higher
overhead than the atomics do, as a vast majority of the time, there
will not be writers.
This moves us further away from using the transport interface, which
will be removed. Each writer should be self contained, without a
separate open or available function.
Also, keep the pmsg fd open if it is opened by
__android_log_pmsg_file_write(). This fd was closed due to issues
with zygote, but it looks like it is only called by recovery now, so
there is no reason to close this fd at the end of that function.
Test: logging works, liblog-unit-tests
Change-Id: I345c9a5d18c55b11a280c8362df854784abf46fd
diff --git a/liblog/fake_log_device.cpp b/liblog/fake_log_device.cpp
index 4143fa6..1fb88e3 100644
--- a/liblog/fake_log_device.cpp
+++ b/liblog/fake_log_device.cpp
@@ -48,21 +48,20 @@
#define TRACE(...) ((void)0)
#endif
-static int FakeAvailable(log_id_t);
-static int FakeOpen();
static void FakeClose();
static int FakeWrite(log_id_t log_id, struct timespec* ts, struct iovec* vec, size_t nr);
struct android_log_transport_write fakeLoggerWrite = {
.name = "fake",
.logMask = 0,
- .available = FakeAvailable,
- .open = FakeOpen,
+ .available = [](log_id_t) { return 0; },
+ .open = [] { return 0; },
.close = FakeClose,
.write = FakeWrite,
};
typedef struct LogState {
+ bool initialized = false;
/* global minimum priority */
int global_min_priority;
@@ -76,19 +75,8 @@
} tagSet[kTagSetSize];
} LogState;
-/*
- * Locking. Since we're emulating a device, we need to be prepared
- * to have multiple callers at the same time. This lock is used
- * to both protect the fd list and to prevent LogStates from being
- * freed out from under a user.
- */
-std::mutex mutex;
-
static LogState log_state;
-
-static int FakeAvailable(log_id_t) {
- return 0;
-}
+static std::mutex fake_log_mutex;
/*
* Configure logging based on ANDROID_LOG_TAGS environment variable. We
@@ -103,8 +91,8 @@
* We also want to check ANDROID_PRINTF_LOG to determine how the output
* will look.
*/
-int FakeOpen() {
- std::lock_guard guard{mutex};
+void InitializeLogStateLocked() {
+ log_state.initialized = true;
/* global min priority defaults to "info" level */
log_state.global_min_priority = ANDROID_LOG_INFO;
@@ -129,7 +117,7 @@
}
if (i == kMaxTagLen) {
TRACE("ERROR: env tag too long (%d chars max)\n", kMaxTagLen - 1);
- return 0;
+ return;
}
tagName[i] = '\0';
@@ -180,7 +168,7 @@
if (*tags != '\0' && !isspace(*tags)) {
TRACE("ERROR: garbage in tag env; expected whitespace\n");
TRACE(" env='%s'\n", tags);
- return 0;
+ return;
}
}
@@ -224,7 +212,6 @@
}
log_state.output_format = format;
- return 0;
}
/*
@@ -474,7 +461,11 @@
* Also guarantees that only one thread is in showLog() at a given
* time (if it matters).
*/
- std::lock_guard guard{mutex};
+ auto lock = std::lock_guard{fake_log_mutex};
+
+ if (!log_state.initialized) {
+ InitializeLogStateLocked();
+ }
if (log_id == LOG_ID_EVENTS || log_id == LOG_ID_STATS || log_id == LOG_ID_SECURITY) {
TRACE("%s: ignoring binary log\n", android_log_id_to_name(log_id));
@@ -532,7 +523,7 @@
* help debug HOST tools ...
*/
static void FakeClose() {
- std::lock_guard guard{mutex};
+ auto lock = std::lock_guard{fake_log_mutex};
memset(&log_state, 0, sizeof(log_state));
}
diff --git a/liblog/logd_writer.cpp b/liblog/logd_writer.cpp
index a22c3be..283a979 100644
--- a/liblog/logd_writer.cpp
+++ b/liblog/logd_writer.cpp
@@ -30,97 +30,80 @@
#include <time.h>
#include <unistd.h>
+#include <shared_mutex>
+
#include <cutils/sockets.h>
#include <private/android_filesystem_config.h>
#include <private/android_logger.h>
#include "log_portability.h"
#include "logger.h"
+#include "rwlock.h"
#include "uio.h"
-static int logdAvailable(log_id_t LogId);
-static int logdOpen();
-static void logdClose();
-static int logdWrite(log_id_t logId, struct timespec* ts, struct iovec* vec, size_t nr);
+static int LogdWrite(log_id_t logId, struct timespec* ts, struct iovec* vec, size_t nr);
+static void LogdClose();
struct android_log_transport_write logdLoggerWrite = {
.name = "logd",
.logMask = 0,
- .context.sock = -EBADF,
- .available = logdAvailable,
- .open = logdOpen,
- .close = logdClose,
- .write = logdWrite,
+ .available = [](log_id_t) { return 0; },
+ .open = [] { return 0; },
+ .close = LogdClose,
+ .write = LogdWrite,
};
-/* log_init_lock assumed */
-static int logdOpen() {
- int i, ret = 0;
+static int logd_socket;
+static RwLock logd_socket_lock;
- i = atomic_load(&logdLoggerWrite.context.sock);
- if (i < 0) {
- int sock = TEMP_FAILURE_RETRY(socket(PF_UNIX, SOCK_DGRAM | SOCK_CLOEXEC | SOCK_NONBLOCK, 0));
- if (sock < 0) {
- ret = -errno;
- } else {
- struct sockaddr_un un;
- memset(&un, 0, sizeof(struct sockaddr_un));
- un.sun_family = AF_UNIX;
- strcpy(un.sun_path, "/dev/socket/logdw");
-
- if (TEMP_FAILURE_RETRY(connect(sock, (struct sockaddr*)&un, sizeof(struct sockaddr_un))) <
- 0) {
- ret = -errno;
- switch (ret) {
- case -ENOTCONN:
- case -ECONNREFUSED:
- case -ENOENT:
- i = atomic_exchange(&logdLoggerWrite.context.sock, ret);
- [[fallthrough]];
- default:
- break;
- }
- close(sock);
- } else {
- ret = atomic_exchange(&logdLoggerWrite.context.sock, sock);
- if ((ret >= 0) && (ret != sock)) {
- close(ret);
- }
- ret = 0;
- }
- }
+static void OpenSocketLocked() {
+ logd_socket = TEMP_FAILURE_RETRY(socket(PF_UNIX, SOCK_DGRAM | SOCK_CLOEXEC | SOCK_NONBLOCK, 0));
+ if (logd_socket <= 0) {
+ return;
}
- return ret;
-}
+ sockaddr_un un = {};
+ un.sun_family = AF_UNIX;
+ strcpy(un.sun_path, "/dev/socket/logdw");
-static void __logdClose(int negative_errno) {
- int sock = atomic_exchange(&logdLoggerWrite.context.sock, negative_errno);
- if (sock >= 0) {
- close(sock);
+ if (TEMP_FAILURE_RETRY(
+ connect(logd_socket, reinterpret_cast<sockaddr*>(&un), sizeof(sockaddr_un))) < 0) {
+ close(logd_socket);
+ logd_socket = 0;
}
}
-static void logdClose() {
- __logdClose(-EBADF);
+static void OpenSocket() {
+ auto lock = std::unique_lock{logd_socket_lock};
+ if (logd_socket > 0) {
+ // Someone raced us and opened the socket already.
+ return;
+ }
+
+ OpenSocketLocked();
}
-static int logdAvailable(log_id_t logId) {
- if (logId >= LOG_ID_MAX || logId == LOG_ID_KERNEL) {
- return -EINVAL;
+static void ResetSocket(int old_socket) {
+ auto lock = std::unique_lock{logd_socket_lock};
+ if (old_socket != logd_socket) {
+ // Someone raced us and reset the socket already.
+ return;
}
- if (atomic_load(&logdLoggerWrite.context.sock) < 0) {
- if (access("/dev/socket/logdw", W_OK) == 0) {
- return 0;
- }
- return -EBADF;
- }
- return 1;
+ close(logd_socket);
+ logd_socket = 0;
+ OpenSocketLocked();
}
-static int logdWrite(log_id_t logId, struct timespec* ts, struct iovec* vec, size_t nr) {
+static void LogdClose() {
+ auto lock = std::unique_lock{logd_socket_lock};
+ if (logd_socket > 0) {
+ close(logd_socket);
+ }
+ logd_socket = 0;
+}
+
+static int LogdWrite(log_id_t logId, struct timespec* ts, struct iovec* vec, size_t nr) {
ssize_t ret;
- int sock;
static const unsigned headerLength = 1;
struct iovec newVec[nr + headerLength];
android_log_header_t header;
@@ -128,15 +111,16 @@
static atomic_int dropped;
static atomic_int droppedSecurity;
- sock = atomic_load(&logdLoggerWrite.context.sock);
- if (sock < 0) switch (sock) {
- case -ENOTCONN:
- case -ECONNREFUSED:
- case -ENOENT:
- break;
- default:
- return -EBADF;
- }
+ auto lock = std::shared_lock{logd_socket_lock};
+ if (logd_socket <= 0) {
+ lock.unlock();
+ OpenSocket();
+ lock.lock();
+ }
+
+ if (logd_socket <= 0) {
+ return -EBADF;
+ }
/* logd, after initialization and priv drop */
if (__android_log_uid() == AID_LOGD) {
@@ -155,41 +139,39 @@
newVec[0].iov_base = (unsigned char*)&header;
newVec[0].iov_len = sizeof(header);
- if (sock >= 0) {
- int32_t snapshot = atomic_exchange_explicit(&droppedSecurity, 0, memory_order_relaxed);
- if (snapshot) {
- android_log_event_int_t buffer;
+ int32_t snapshot = atomic_exchange_explicit(&droppedSecurity, 0, memory_order_relaxed);
+ if (snapshot) {
+ android_log_event_int_t buffer;
- header.id = LOG_ID_SECURITY;
- buffer.header.tag = LIBLOG_LOG_TAG;
- buffer.payload.type = EVENT_TYPE_INT;
- buffer.payload.data = snapshot;
+ header.id = LOG_ID_SECURITY;
+ buffer.header.tag = LIBLOG_LOG_TAG;
+ buffer.payload.type = EVENT_TYPE_INT;
+ buffer.payload.data = snapshot;
- newVec[headerLength].iov_base = &buffer;
- newVec[headerLength].iov_len = sizeof(buffer);
+ newVec[headerLength].iov_base = &buffer;
+ newVec[headerLength].iov_len = sizeof(buffer);
- ret = TEMP_FAILURE_RETRY(writev(sock, newVec, 2));
- if (ret != (ssize_t)(sizeof(header) + sizeof(buffer))) {
- atomic_fetch_add_explicit(&droppedSecurity, snapshot, memory_order_relaxed);
- }
+ ret = TEMP_FAILURE_RETRY(writev(logd_socket, newVec, 2));
+ if (ret != (ssize_t)(sizeof(header) + sizeof(buffer))) {
+ atomic_fetch_add_explicit(&droppedSecurity, snapshot, memory_order_relaxed);
}
- snapshot = atomic_exchange_explicit(&dropped, 0, memory_order_relaxed);
- if (snapshot && __android_log_is_loggable_len(ANDROID_LOG_INFO, "liblog", strlen("liblog"),
- ANDROID_LOG_VERBOSE)) {
- android_log_event_int_t buffer;
+ }
+ snapshot = atomic_exchange_explicit(&dropped, 0, memory_order_relaxed);
+ if (snapshot && __android_log_is_loggable_len(ANDROID_LOG_INFO, "liblog", strlen("liblog"),
+ ANDROID_LOG_VERBOSE)) {
+ android_log_event_int_t buffer;
- header.id = LOG_ID_EVENTS;
- buffer.header.tag = LIBLOG_LOG_TAG;
- buffer.payload.type = EVENT_TYPE_INT;
- buffer.payload.data = snapshot;
+ header.id = LOG_ID_EVENTS;
+ buffer.header.tag = LIBLOG_LOG_TAG;
+ buffer.payload.type = EVENT_TYPE_INT;
+ buffer.payload.data = snapshot;
- newVec[headerLength].iov_base = &buffer;
- newVec[headerLength].iov_len = sizeof(buffer);
+ newVec[headerLength].iov_base = &buffer;
+ newVec[headerLength].iov_len = sizeof(buffer);
- ret = TEMP_FAILURE_RETRY(writev(sock, newVec, 2));
- if (ret != (ssize_t)(sizeof(header) + sizeof(buffer))) {
- atomic_fetch_add_explicit(&dropped, snapshot, memory_order_relaxed);
- }
+ ret = TEMP_FAILURE_RETRY(writev(logd_socket, newVec, 2));
+ if (ret != (ssize_t)(sizeof(header) + sizeof(buffer))) {
+ atomic_fetch_add_explicit(&dropped, snapshot, memory_order_relaxed);
}
}
@@ -208,49 +190,26 @@
}
}
- /*
- * The write below could be lost, but will never block.
- *
- * ENOTCONN occurs if logd has died.
- * ENOENT occurs if logd is not running and socket is missing.
- * ECONNREFUSED occurs if we can not reconnect to logd.
- * EAGAIN occurs if logd is overloaded.
- */
- if (sock < 0) {
- ret = sock;
- } else {
- ret = TEMP_FAILURE_RETRY(writev(sock, newVec, i));
- if (ret < 0) {
- ret = -errno;
- }
+ // The write below could be lost, but will never block.
+ // EAGAIN occurs if logd is overloaded, other errors indicate that something went wrong with
+ // the connection, so we reset it and try again.
+ ret = TEMP_FAILURE_RETRY(writev(logd_socket, newVec, i));
+ if (ret < 0 && errno != EAGAIN) {
+ int old_socket = logd_socket;
+ lock.unlock();
+ ResetSocket(old_socket);
+ lock.lock();
+
+ ret = TEMP_FAILURE_RETRY(writev(logd_socket, newVec, i));
}
- switch (ret) {
- case -ENOTCONN:
- case -ECONNREFUSED:
- case -ENOENT:
- if (__android_log_trylock()) {
- return ret; /* in a signal handler? try again when less stressed */
- }
- __logdClose(ret);
- ret = logdOpen();
- __android_log_unlock();
- if (ret < 0) {
- return ret;
- }
-
- ret = TEMP_FAILURE_RETRY(writev(atomic_load(&logdLoggerWrite.context.sock), newVec, i));
- if (ret < 0) {
- ret = -errno;
- }
- [[fallthrough]];
- default:
- break;
+ if (ret < 0) {
+ ret = -errno;
}
if (ret > (ssize_t)sizeof(header)) {
ret -= sizeof(header);
- } else if (ret == -EAGAIN) {
+ } else if (ret < 0) {
atomic_fetch_add_explicit(&dropped, 1, memory_order_relaxed);
if (logId == LOG_ID_SECURITY) {
atomic_fetch_add_explicit(&droppedSecurity, 1, memory_order_relaxed);
diff --git a/liblog/pmsg_writer.cpp b/liblog/pmsg_writer.cpp
index 54980d9..de48086 100644
--- a/liblog/pmsg_writer.cpp
+++ b/liblog/pmsg_writer.cpp
@@ -25,68 +25,51 @@
#include <sys/types.h>
#include <time.h>
+#include <shared_mutex>
+
#include <log/log_properties.h>
#include <private/android_filesystem_config.h>
#include <private/android_logger.h>
#include "log_portability.h"
#include "logger.h"
+#include "rwlock.h"
#include "uio.h"
-static int pmsgOpen();
-static void pmsgClose();
-static int pmsgAvailable(log_id_t logId);
-static int pmsgWrite(log_id_t logId, struct timespec* ts, struct iovec* vec, size_t nr);
+static void PmsgClose();
+static int PmsgWrite(log_id_t logId, struct timespec* ts, struct iovec* vec, size_t nr);
struct android_log_transport_write pmsgLoggerWrite = {
.name = "pmsg",
.logMask = 0,
- .context.fd = -1,
- .available = pmsgAvailable,
- .open = pmsgOpen,
- .close = pmsgClose,
- .write = pmsgWrite,
+ .available = [](log_id_t) { return 0; },
+ .open = [] { return 0; },
+ .close = PmsgClose,
+ .write = PmsgWrite,
};
-static int pmsgOpen() {
- int fd = atomic_load(&pmsgLoggerWrite.context.fd);
- if (fd < 0) {
- int i;
+static int pmsg_fd;
+static RwLock pmsg_fd_lock;
- fd = TEMP_FAILURE_RETRY(open("/dev/pmsg0", O_WRONLY | O_CLOEXEC));
- i = atomic_exchange(&pmsgLoggerWrite.context.fd, fd);
- if ((i >= 0) && (i != fd)) {
- close(i);
- }
+static void PmsgOpen() {
+ auto lock = std::unique_lock{pmsg_fd_lock};
+ if (pmsg_fd > 0) {
+ // Someone raced us and opened the socket already.
+ return;
}
- return fd;
+ pmsg_fd = TEMP_FAILURE_RETRY(open("/dev/pmsg0", O_WRONLY | O_CLOEXEC));
}
-static void pmsgClose() {
- int fd = atomic_exchange(&pmsgLoggerWrite.context.fd, -1);
- if (fd >= 0) {
- close(fd);
+static void PmsgClose() {
+ auto lock = std::unique_lock{pmsg_fd_lock};
+ if (pmsg_fd > 0) {
+ close(pmsg_fd);
}
+ pmsg_fd = 0;
}
-static int pmsgAvailable(log_id_t logId) {
- if (logId > LOG_ID_SECURITY) {
- return -EINVAL;
- }
- if ((logId != LOG_ID_SECURITY) && (logId != LOG_ID_EVENTS) && !__android_log_is_debuggable()) {
- return -EINVAL;
- }
- if (atomic_load(&pmsgLoggerWrite.context.fd) < 0) {
- if (access("/dev/pmsg0", W_OK) == 0) {
- return 0;
- }
- return -EBADF;
- }
- return 1;
-}
-
-static int pmsgWrite(log_id_t logId, struct timespec* ts, struct iovec* vec, size_t nr) {
+static int PmsgWrite(log_id_t logId, struct timespec* ts, struct iovec* vec, size_t nr) {
static const unsigned headerLength = 2;
struct iovec newVec[nr + headerLength];
android_log_header_t header;
@@ -94,17 +77,31 @@
size_t i, payloadSize;
ssize_t ret;
- if ((logId == LOG_ID_EVENTS) && !__android_log_is_debuggable()) {
- if (vec[0].iov_len < 4) {
- return -EINVAL;
+ if (!__android_log_is_debuggable()) {
+ if (logId != LOG_ID_EVENTS && logId != LOG_ID_SECURITY) {
+ return -1;
}
- if (SNET_EVENT_LOG_TAG != *static_cast<uint32_t*>(vec[0].iov_base)) {
- return -EPERM;
+ if (logId == LOG_ID_EVENTS) {
+ if (vec[0].iov_len < 4) {
+ return -EINVAL;
+ }
+
+ if (SNET_EVENT_LOG_TAG != *static_cast<uint32_t*>(vec[0].iov_base)) {
+ return -EPERM;
+ }
}
}
- if (atomic_load(&pmsgLoggerWrite.context.fd) < 0) {
+ auto lock = std::shared_lock{pmsg_fd_lock};
+
+ if (pmsg_fd <= 0) {
+ lock.unlock();
+ PmsgOpen();
+ lock.lock();
+ }
+
+ if (pmsg_fd <= 0) {
return -EBADF;
}
@@ -158,7 +155,7 @@
}
pmsgHeader.len += payloadSize;
- ret = TEMP_FAILURE_RETRY(writev(atomic_load(&pmsgLoggerWrite.context.fd), newVec, i));
+ ret = TEMP_FAILURE_RETRY(writev(pmsg_fd, newVec, i));
if (ret < 0) {
ret = errno ? -errno : -ENOTCONN;
}
@@ -193,7 +190,6 @@
/* Write a buffer as filename references (tag = <basedir>:<basename>) */
ssize_t __android_log_pmsg_file_write(log_id_t logId, char prio, const char* filename,
const char* buf, size_t len) {
- bool weOpened;
size_t length, packet_len;
const char* tag;
char *cp, *slash;
@@ -233,7 +229,6 @@
vec[1].iov_base = (unsigned char*)tag;
vec[1].iov_len = length;
- weOpened = false;
for (ts.tv_nsec = 0, length = len; length; ts.tv_nsec += ANDROID_LOG_PMSG_FILE_SEQUENCE) {
ssize_t ret;
size_t transfer;
@@ -254,37 +249,15 @@
vec[2].iov_base = (unsigned char*)buf;
vec[2].iov_len = transfer;
- if (atomic_load(&pmsgLoggerWrite.context.fd) < 0) {
- if (!weOpened) { /* Impossible for weOpened = true here */
- __android_log_lock();
- }
- weOpened = atomic_load(&pmsgLoggerWrite.context.fd) < 0;
- if (!weOpened) {
- __android_log_unlock();
- } else if (pmsgOpen() < 0) {
- __android_log_unlock();
- free(cp);
- return -EBADF;
- }
- }
-
- ret = pmsgWrite(logId, &ts, vec, sizeof(vec) / sizeof(vec[0]));
+ ret = PmsgWrite(logId, &ts, vec, sizeof(vec) / sizeof(vec[0]));
if (ret <= 0) {
- if (weOpened) {
- pmsgClose();
- __android_log_unlock();
- }
free(cp);
return ret ? ret : (len - length);
}
length -= transfer;
buf += transfer;
}
- if (weOpened) {
- pmsgClose();
- __android_log_unlock();
- }
free(cp);
return len;
}
diff --git a/liblog/rwlock.h b/liblog/rwlock.h
new file mode 100644
index 0000000..00f1806
--- /dev/null
+++ b/liblog/rwlock.h
@@ -0,0 +1,39 @@
+/*
+ * Copyright (C) 2019 The Android Open Source Project
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <pthread.h>
+
+// As of the end of Dec 2019, std::shared_mutex is *not* simply a pthread_rwlock, but rather a
+// combination of std::mutex and std::condition variable, which is obviously less efficient. This
+// immitates what std::shared_mutex should be doing and is compatible with std::shared_lock and
+// std::unique_lock.
+
+class RwLock {
+ public:
+ RwLock() {}
+ ~RwLock() {}
+
+ void lock() { pthread_rwlock_wrlock(&rwlock_); }
+ void unlock() { pthread_rwlock_unlock(&rwlock_); }
+
+ void lock_shared() { pthread_rwlock_rdlock(&rwlock_); }
+ void unlock_shared() { pthread_rwlock_unlock(&rwlock_); }
+
+ private:
+ pthread_rwlock_t rwlock_ = PTHREAD_RWLOCK_INITIALIZER;
+};
diff --git a/liblog/tests/Android.bp b/liblog/tests/Android.bp
index 99df4ca..f58c524 100644
--- a/liblog/tests/Android.bp
+++ b/liblog/tests/Android.bp
@@ -62,6 +62,7 @@
"log_time_test.cpp",
"log_wrap_test.cpp",
"logprint_test.cpp",
+ "rwlock_test.cpp",
],
shared_libs: [
"libcutils",
diff --git a/liblog/tests/rwlock_test.cpp b/liblog/tests/rwlock_test.cpp
new file mode 100644
index 0000000..617d5c4
--- /dev/null
+++ b/liblog/tests/rwlock_test.cpp
@@ -0,0 +1,91 @@
+/*
+ * Copyright (C) 2019 The Android Open Source Project
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "../rwlock.h"
+
+#include <chrono>
+#include <shared_mutex>
+#include <thread>
+
+#include <gtest/gtest.h>
+
+using namespace std::literals;
+
+TEST(rwlock, reader_then_reader_lock) {
+ RwLock lock;
+
+ bool thread_ran = false;
+ auto read_guard = std::shared_lock{lock};
+
+ auto reader_thread = std::thread([&] {
+ auto read_guard = std::shared_lock{lock};
+ thread_ran = true;
+ });
+
+ auto end_time = std::chrono::steady_clock::now() + 1s;
+
+ while (std::chrono::steady_clock::now() < end_time) {
+ if (thread_ran) {
+ break;
+ }
+ }
+
+ EXPECT_EQ(true, thread_ran);
+
+ // Unlock the lock in case something went wrong, to ensure that we can still join() the thread.
+ read_guard.unlock();
+ reader_thread.join();
+}
+
+template <template <typename> typename L1, template <typename> typename L2>
+void TestBlockingLocks() {
+ RwLock lock;
+
+ bool thread_ran = false;
+ auto read_guard = L1{lock};
+
+ auto reader_thread = std::thread([&] {
+ auto read_guard = L2{lock};
+ thread_ran = true;
+ });
+
+ auto end_time = std::chrono::steady_clock::now() + 1s;
+
+ while (std::chrono::steady_clock::now() < end_time) {
+ if (thread_ran) {
+ break;
+ }
+ }
+
+ EXPECT_EQ(false, thread_ran);
+
+ read_guard.unlock();
+ reader_thread.join();
+
+ EXPECT_EQ(true, thread_ran);
+}
+
+TEST(rwlock, reader_then_writer_lock) {
+ TestBlockingLocks<std::shared_lock, std::unique_lock>();
+}
+
+TEST(rwlock, writer_then_reader_lock) {
+ TestBlockingLocks<std::unique_lock, std::shared_lock>();
+}
+
+TEST(rwlock, writer_then_writer_lock) {
+ TestBlockingLocks<std::unique_lock, std::unique_lock>();
+}