This is an automated email from the ASF dual-hosted git repository.

masaori pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/trafficserver.git


The following commit(s) were added to refs/heads/master by this push:
     new c331c7205c Add BRAVO Reader-Writer Lock (#9394)
c331c7205c is described below

commit c331c7205cf1b0f4a4955211ccbec50d36fb063e
Author: Masaori Koshiba <masa...@apache.org>
AuthorDate: Tue May 23 07:52:57 2023 +0900

    Add BRAVO Reader-Writer Lock (#9394)
    
    * Add BRAVO Reader-Writer Lock
    
    * Add DenseThreadId
    
    Signed-off-by: Walt Karas <wka...@yahooinc.com>
---
 .gitignore                                      |   1 +
 NOTICE                                          |   5 +
 include/tscpp/util/Bravo.h                      | 377 ++++++++++++++++++++++++
 include/tscpp/util/DenseThreadId.h              | 116 ++++++++
 src/tscore/Makefile.am                          |   7 +-
 src/tscore/unit_tests/benchmark_shared_mutex.cc | 134 +++++++++
 src/tscore/unit_tests/test_Bravo.cc             | 229 ++++++++++++++
 src/tscpp/util/Makefile.am                      |   4 +-
 8 files changed, 871 insertions(+), 2 deletions(-)

diff --git a/.gitignore b/.gitignore
index 30a42adcc6..426124c8ef 100644
--- a/.gitignore
+++ b/.gitignore
@@ -86,6 +86,7 @@ src/tscore/ink_autoconf.h
 src/tscore/ink_autoconf.h.in
 include/tscore/ink_config.h
 include/ts/apidefs.h
+src/tscore/benchmark_shared_mutex
 src/tscore/CompileParseRules
 src/tscore/CompileParseRules.dSYM
 src/tscore/ParseRulesCType
diff --git a/NOTICE b/NOTICE
index 949f852011..50552702ca 100644
--- a/NOTICE
+++ b/NOTICE
@@ -96,3 +96,8 @@ https://github.com/jbeder/yaml-cpp
 
 fastlz: an ANSI C/C90 implementation of Lempel-Ziv 77 algorithm (LZ77) of 
lossless data compression.
 https://github.com/ariya/FastLZ
+
+~~
+
+include/tscpp/util/Bravo.h is C++ version of puzpuzpuz/xsync's RBMutex
+Copyright (c) 2021 Andrey Pechkurov (MIT License)
diff --git a/include/tscpp/util/Bravo.h b/include/tscpp/util/Bravo.h
new file mode 100644
index 0000000000..a9155d3492
--- /dev/null
+++ b/include/tscpp/util/Bravo.h
@@ -0,0 +1,377 @@
+/** @file
+
+  Implementation of BRAVO - Biased Locking for Reader-Writer Locks
+
+  Dave Dice and Alex Kogan. 2019. BRAVO: Biased Locking for Reader-Writer 
Locks.
+  In Proceedings of the 2019 USENIX Annual Technical Conference (ATC). USENIX 
Association, Renton, WA, 315–328.
+
+  https://www.usenix.org/conference/atc19/presentation/dice
+
+  > Section 3.
+  >   BRAVO acts as an accelerator layer, as readers can always fall back to 
the traditional underlying lock to gain read access.
+  >   ...
+  >   Write performance and the scalability of read-vs-write and 
write-vs-write behavior depends solely on the underlying lock.
+
+  This code is C++ version of puzpuzpuz/xsync's RBMutex
+  https://github.com/puzpuzpuz/xsync/blob/main/rbmutex.go
+  Copyright (c) 2021 Andrey Pechkurov
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you 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 "DenseThreadId.h"
+
+#include "tscore/Diags.h"
+#include "tscore/ink_assert.h"
+
+#include <array>
+#include <atomic>
+#include <cassert>
+#include <chrono>
+#include <shared_mutex>
+#include <thread>
+
+namespace ts::bravo
+{
+using time_point = std::chrono::time_point<std::chrono::system_clock>;
+
+#ifdef __cpp_lib_hardware_interference_size
+using std::hardware_constructive_interference_size;
+#else
+// 64 bytes on x86-64 │ L1_CACHE_BYTES │ L1_CACHE_SHIFT │ __cacheline_aligned 
│ ...
+constexpr std::size_t hardware_constructive_interference_size = 64;
+#endif
+
+/**
+   ts::bravo::Token
+
+   Token for readers.
+   0 is special value that represents inital/invalid value.
+ */
+using Token = size_t;
+
+/**
+   ts::bravo::shared_lock
+ */
+template <class Mutex> class shared_lock
+{
+public:
+  using mutex_type = Mutex;
+
+  shared_lock() noexcept = default;
+  shared_lock(Mutex &m) : _mutex(&m) { lock(); }
+  shared_lock(Mutex &m, std::try_to_lock_t) : _mutex(&m) { try_lock(); }
+  shared_lock(Mutex &m, std::defer_lock_t) noexcept : _mutex(&m) {}
+
+  ~shared_lock()
+  {
+    if (_owns) {
+      _mutex->unlock_shared(_token);
+    }
+  };
+
+  ////
+  // Not Copyable
+  //
+  shared_lock(shared_lock const &)            = delete;
+  shared_lock &operator=(shared_lock const &) = delete;
+
+  ////
+  // Moveable
+  //
+  shared_lock(shared_lock &&s) : _mutex(s._mutex), _token(s._token), 
_owns(s._owns)
+  {
+    s._mutex = nullptr;
+    s._token = 0;
+    s._owns  = false;
+  };
+
+  shared_lock &
+  operator=(shared_lock &&s)
+  {
+    if (_owns) {
+      _mutex->unlock_shared(_token);
+    }
+    _mutex = s._mutex;
+    _token = s._token;
+    _owns  = s._owns;
+
+    s._mutex = nullptr;
+    s._token = 0;
+    s._owns  = false;
+  };
+
+  ////
+  // Shared locking
+  //
+  void
+  lock()
+  {
+    _mutex->lock_shared(_token);
+    _owns = true;
+  }
+
+  bool
+  try_lock()
+  {
+    _owns = _mutex->try_lock_shared(_token);
+    return _owns;
+  }
+
+  // not implemented yet
+  bool try_lock_for()   = delete;
+  bool try_lock_until() = delete;
+
+  void
+  unlock()
+  {
+    _mutex->unlock_shared(_token);
+    _owns = false;
+  }
+
+  ////
+  // Modifiers
+  //
+  void
+  swap(shared_lock &s)
+  {
+    std::swap(_mutex, s._mutex);
+    std::swap(_token, s._token);
+    std::swap(_owns, s._owns);
+  }
+
+  mutex_type *
+  release()
+  {
+    mutex_type *m = _mutex;
+    _mutex        = nullptr;
+    _token        = 0;
+    _owns         = false;
+    return m;
+  }
+
+  ////
+  // Observers
+  //
+  mutex_type *
+  mutex()
+  {
+    return _mutex;
+  }
+
+  Token
+  token()
+  {
+    return _token;
+  }
+
+  bool
+  owns_lock()
+  {
+    return _owns;
+  }
+
+private:
+  mutex_type *_mutex = nullptr;
+  Token _token       = 0;
+  bool _owns         = false;
+};
+
+/**
+   ts::bravo::shared_mutex
+
+   You can use std::lock_guard for writers but, you can't use std::shared_lock 
for readers to handle ts::bravo::Token.
+   Use ts::bravo::shared_lock for readers.
+
+   Set the SLOT_SIZE larger than DenseThreadId::num_possible_values to go 
fast-path.
+ */
+template <typename T = std::shared_mutex, size_t SLOT_SIZE = 256, int 
SLOWDOWN_GUARD = 7> class shared_mutex_impl
+{
+public:
+  shared_mutex_impl()  = default;
+  ~shared_mutex_impl() = default;
+
+  ////
+  // No copying or moving.
+  //
+  shared_mutex_impl(shared_mutex_impl const &)            = delete;
+  shared_mutex_impl &operator=(shared_mutex_impl const &) = delete;
+
+  shared_mutex_impl(shared_mutex_impl &&)            = delete;
+  shared_mutex_impl &operator=(shared_mutex_impl &&) = delete;
+
+  ////
+  // Exclusive locking
+  //
+  void
+  lock()
+  {
+    _mutex.underlying.lock();
+    _revoke();
+  }
+
+  bool
+  try_lock()
+  {
+    bool r = _mutex.underlying.try_lock();
+    if (!r) {
+      return false;
+    }
+
+    _revoke();
+
+    return true;
+  }
+
+  void
+  unlock()
+  {
+    _mutex.underlying.unlock();
+  }
+
+  ////
+  // Shared locking
+  //
+  void
+  lock_shared(Token &token)
+  {
+    ink_assert(SLOT_SIZE >= DenseThreadId::num_possible_values());
+
+    // Fast path
+    if (_mutex.read_bias.load(std::memory_order_acquire)) {
+      size_t index = DenseThreadId::self() % SLOT_SIZE;
+      Slot &slot   = _mutex.readers[index];
+      bool expect  = false;
+      if (slot.mu.compare_exchange_strong(expect, true, 
std::memory_order_relaxed)) {
+        // recheck
+        if (_mutex.read_bias.load(std::memory_order_acquire)) {
+          token = index + 1;
+          return;
+        } else {
+          slot.mu.store(false, std::memory_order_relaxed);
+        }
+      }
+    }
+
+    // Slow path
+    _mutex.underlying.lock_shared();
+    if (_mutex.read_bias.load(std::memory_order_acquire) == false && _now() >= 
_mutex.inhibit_until) {
+      _mutex.read_bias.store(true, std::memory_order_release);
+    }
+  }
+
+  bool
+  try_lock_shared(Token &token)
+  {
+    ink_assert(SLOT_SIZE >= DenseThreadId::num_possible_values());
+
+    // Fast path
+    if (_mutex.read_bias.load(std::memory_order_acquire)) {
+      size_t index = DenseThreadId::self() % SLOT_SIZE;
+      Slot &slot   = _mutex.readers[index];
+      bool expect  = false;
+
+      if (slot.mu.compare_exchange_weak(expect, true, 
std::memory_order_release, std::memory_order_relaxed)) {
+        // recheck
+        if (_mutex.read_bias.load(std::memory_order_acquire)) {
+          token = index + 1;
+          return true;
+        } else {
+          slot.mu.store(false, std::memory_order_relaxed);
+        }
+      }
+    }
+
+    // Slow path
+    bool r = _mutex.underlying.try_lock_shared();
+    if (r) {
+      // Set RBias if the BRAVO policy allows that
+      if (_mutex.read_bias.load(std::memory_order_acquire) == false && _now() 
>= _mutex.inhibit_until) {
+        _mutex.read_bias.store(true, std::memory_order_release);
+      }
+
+      return true;
+    }
+
+    return false;
+  }
+
+  void
+  unlock_shared(const Token token)
+  {
+    if (token == 0) {
+      _mutex.underlying.unlock_shared();
+      return;
+    }
+
+    Slot &slot = _mutex.readers[token - 1];
+    slot.mu.store(false, std::memory_order_relaxed);
+  }
+
+private:
+  struct alignas(hardware_constructive_interference_size) Slot {
+    std::atomic<bool> mu = false;
+  };
+
+  struct Mutex {
+    std::atomic<bool> read_bias         = false;
+    std::array<Slot, SLOT_SIZE> readers = {};
+    time_point inhibit_until{};
+    T underlying;
+  };
+
+  time_point
+  _now()
+  {
+    return std::chrono::system_clock::now();
+  }
+
+  /**
+     Disable read bias and do revocation
+   */
+  void
+  _revoke()
+  {
+    if (!_mutex.read_bias.load(std::memory_order_acquire)) {
+      // do nothing
+      return;
+    }
+
+    _mutex.read_bias.store(false, std::memory_order_release);
+    time_point start = _now();
+    for (size_t i = 0; i < SLOT_SIZE; ++i) {
+      for (int j = 0; _mutex.readers[i].mu.load(std::memory_order_relaxed); 
++j) {
+        std::this_thread::sleep_for(std::chrono::nanoseconds(1 << j));
+      }
+    }
+    time_point n         = _now();
+    _mutex.inhibit_until = n + ((n - start) * SLOWDOWN_GUARD);
+  }
+
+  ////
+  // Variables
+  //
+  Mutex _mutex;
+};
+
+using shared_mutex = shared_mutex_impl<>;
+
+} // namespace ts::bravo
diff --git a/include/tscpp/util/DenseThreadId.h 
b/include/tscpp/util/DenseThreadId.h
new file mode 100644
index 0000000000..9189cf7428
--- /dev/null
+++ b/include/tscpp/util/DenseThreadId.h
@@ -0,0 +1,116 @@
+/** @file
+
+  A replacement for std::shared_mutex with guarantees against writer 
starvation.
+  Cache contention between CPU cores is avoided except when a write lock is 
taken.
+  Assumes no thread will exit while holding mutex.
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you 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
+
+#if __has_include(<tscore/ink_assert.h>)
+// Included in core.
+#include <tscore/ink_assert.h>
+#define L_Assert ink_assert
+#include <tscore/Diags.h>
+#define L_Fatal Fatal
+#else
+// Should be plugin code.
+#include <ts/ts.h>
+#define L_Assert TSAssert
+#define L_Fatal  TSFatal
+#endif
+
+#include <cstddef>
+#include <mutex>
+#include <vector>
+
+// Provide an alternate thread id, suitible for use as an array index.
+//
+class DenseThreadId
+{
+public:
+  // This can onlhy be called during single-threaded initialization.
+  //
+  static void
+  set_num_possible_values(std::size_t num_possible_values)
+  {
+    _num_possible_values = num_possible_values;
+  }
+
+  static std::size_t
+  self()
+  {
+    return _id.val;
+  }
+
+  static std::size_t
+  num_possible_values()
+  {
+    return _num_possible_values;
+  }
+
+private:
+  inline static std::mutex _mtx;
+  inline static std::vector<std::size_t> _id_stack;
+  inline static std::size_t _stack_top_idx;
+  inline static std::size_t _num_possible_values{256};
+
+  static void
+  _init()
+  {
+    _id_stack.resize(_num_possible_values);
+
+    _stack_top_idx = 0;
+    for (std::size_t i{0}; i < _num_possible_values; ++i) {
+      _id_stack[i] = i + 1;
+    }
+  }
+
+  struct _Id {
+    _Id()
+    {
+      std::unique_lock<std::mutex> ul{_mtx};
+
+      if (!_inited) {
+        _init();
+        _inited = true;
+      }
+      if (_id_stack.size() == _stack_top_idx) {
+        L_Fatal("DenseThreadId:  number of threads exceeded maximum (%u)", 
unsigned(_id_stack.size()));
+      }
+      val            = _stack_top_idx;
+      _stack_top_idx = _id_stack[_stack_top_idx];
+    }
+
+    ~_Id()
+    {
+      std::unique_lock<std::mutex> ul{_mtx};
+
+      _id_stack[val] = _stack_top_idx;
+      _stack_top_idx = val;
+    }
+
+    std::size_t val;
+  };
+
+  inline static thread_local _Id _id;
+  inline static bool _inited{false};
+};
diff --git a/src/tscore/Makefile.am b/src/tscore/Makefile.am
index 142d6f63d0..5a5f0ef1ae 100644
--- a/src/tscore/Makefile.am
+++ b/src/tscore/Makefile.am
@@ -18,7 +18,7 @@
 
 include $(top_srcdir)/build/tidy.mk
 
-noinst_PROGRAMS = CompileParseRules freelist_benchmark
+noinst_PROGRAMS = CompileParseRules freelist_benchmark benchmark_shared_mutex
 check_PROGRAMS = test_geometry test_X509HostnameValidator test_tscore
 
 if EXPENSIVE_TESTS
@@ -172,6 +172,7 @@ test_tscore_SOURCES = \
        unit_tests/test_ArgParser.cc \
        unit_tests/test_BufferWriter.cc \
        unit_tests/test_BufferWriterFormat.cc \
+       unit_tests/test_Bravo.cc \
        unit_tests/test_CryptoHash.cc \
        unit_tests/test_Extendible.cc \
        unit_tests/test_Histogram.cc \
@@ -205,6 +206,10 @@ freelist_benchmark_CXXFLAGS = -Wno-array-bounds 
$(AM_CXXFLAGS) -I$(abs_top_srcdi
 freelist_benchmark_LDADD = libtscore.la @HWLOC_LIBS@
 freelist_benchmark_SOURCES = unit_tests/freelist_benchmark.cc
 
+benchmark_shared_mutex_CXXFLAGS = -Wno-array-bounds $(AM_CXXFLAGS) 
-I$(abs_top_srcdir)/tests/include
+benchmark_shared_mutex_LDADD = libtscore.la
+benchmark_shared_mutex_SOURCES = unit_tests/benchmark_shared_mutex.cc
+
 CompileParseRules_SOURCES = CompileParseRules.cc
 
 CompileParseRules$(BUILD_EXEEXT): $(CompileParseRules_OBJECTS)
diff --git a/src/tscore/unit_tests/benchmark_shared_mutex.cc 
b/src/tscore/unit_tests/benchmark_shared_mutex.cc
new file mode 100644
index 0000000000..fe1a499d77
--- /dev/null
+++ b/src/tscore/unit_tests/benchmark_shared_mutex.cc
@@ -0,0 +1,134 @@
+/** @file
+
+  Micro Benchmark tool for shared_mutex - requires Catch2 v2.9.0+
+
+  - e.g. example of running 64 threads with read/write rate is 100:1
+  ```
+  $ taskset -c 0-63 ./benchmark_shared_mutex --ts-nthreads 64 --ts-nloop 1000 
--ts-nread 100 --ts-nwrite 1
+  ```
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you 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.
+ */
+
+#define CATCH_CONFIG_ENABLE_BENCHMARKING
+#define CATCH_CONFIG_RUNNER
+
+#include "catch.hpp"
+
+#include "tscpp/util/Bravo.h"
+
+#include <mutex>
+#include <shared_mutex>
+#include <thread>
+
+namespace
+{
+// Args
+struct Conf {
+  int nloop    = 1;
+  int nthreads = 1;
+  int nread    = 1;
+  int nwrite   = 1;
+};
+
+Conf conf;
+template <typename T, typename S>
+int
+run(T &mutex)
+{
+  std::thread list[conf.nthreads];
+  int counter = 0;
+
+  for (int i = 0; i < conf.nthreads; i++) {
+    new (&list[i]) std::thread{[&counter](T &mutex) {
+                                 int c = 0;
+                                 for (int j = 0; j < conf.nloop; ++j) {
+                                   // reader
+                                   for (int i = 0; i < conf.nread; ++i) {
+                                     S lock(mutex);
+                                     // Do not optimize
+                                     c = counter;
+                                   }
+
+                                   // writer
+                                   for (int i = 0; i < conf.nwrite; ++i) {
+                                     std::lock_guard lock(mutex);
+                                     // Do not optimize
+                                     ++c;
+                                     counter = c;
+                                   }
+                                 }
+                               },
+                               std::ref(mutex)};
+  }
+
+  for (int i = 0; i < conf.nthreads; i++) {
+    list[i].join();
+  }
+
+  return counter;
+}
+
+} // namespace
+
+TEST_CASE("Micro benchmark of shared_mutex", "")
+{
+  SECTION("std::shared_mutex")
+  {
+    BENCHMARK("std::shared_mutex", )
+    {
+      std::shared_mutex mutex;
+
+      return run<std::shared_mutex, 
std::shared_lock<std::shared_mutex>>(mutex);
+    };
+  }
+
+  SECTION("ts::bravo::shared_mutex")
+  {
+    BENCHMARK("ts::bravo::shared_mutex")
+    {
+      ts::bravo::shared_mutex mutex;
+
+      return run<ts::bravo::shared_mutex, 
ts::bravo::shared_lock<ts::bravo::shared_mutex>>(mutex);
+    };
+  }
+}
+
+int
+main(int argc, char *argv[])
+{
+  Catch::Session session;
+
+  using namespace Catch::clara;
+
+  // clang-format off
+  auto cli = session.cli() |
+    Opt(conf.nthreads, "")["--ts-nthreads"]("number of threads (default: 1)") |
+    Opt(conf.nread, "")["--ts-nread"]("number of read op (default: 1)") |
+    Opt(conf.nwrite, "")["--ts-nwrite"]("number of write op (default: 1)") |
+    Opt(conf.nloop, "")["--ts-nloop"]("number of read-write loop (default: 
1)");
+  // clang-format on
+
+  session.cli(cli);
+
+  int returnCode = session.applyCommandLine(argc, argv);
+  if (returnCode != 0) {
+    return returnCode;
+  }
+
+  return session.run();
+}
diff --git a/src/tscore/unit_tests/test_Bravo.cc 
b/src/tscore/unit_tests/test_Bravo.cc
new file mode 100644
index 0000000000..e64e7514f3
--- /dev/null
+++ b/src/tscore/unit_tests/test_Bravo.cc
@@ -0,0 +1,229 @@
+/** @file
+
+    Unit tests for BRAVO
+
+    @section license License
+
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you 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 "catch.hpp"
+#include "tscpp/util/Bravo.h"
+
+#include <chrono>
+#include <mutex>
+#include <shared_mutex>
+#include <thread>
+
+using namespace std::chrono_literals;
+
+TEST_CASE("BRAVO - simple check", "[libts][BRAVO]")
+{
+  SECTION("reader-reader")
+  {
+    ts::bravo::shared_mutex mutex;
+    ts::bravo::shared_lock<ts::bravo::shared_mutex> lock(mutex);
+    CHECK(lock.owns_lock() == true);
+
+    std::thread t{[](ts::bravo::shared_mutex &mutex) {
+                    ts::bravo::Token token{0};
+                    CHECK(mutex.try_lock_shared(token) == true);
+                    mutex.unlock_shared(token);
+                  },
+                  std::ref(mutex)};
+
+    t.join();
+  }
+
+  SECTION("reader-writer")
+  {
+    ts::bravo::shared_mutex mutex;
+    ts::bravo::shared_lock<ts::bravo::shared_mutex> lock(mutex);
+    CHECK(lock.owns_lock() == true);
+
+    std::thread t{[](ts::bravo::shared_mutex &mutex) { CHECK(mutex.try_lock() 
== false); }, std::ref(mutex)};
+
+    t.join();
+  }
+
+  SECTION("writer-reader")
+  {
+    ts::bravo::shared_mutex mutex;
+    std::lock_guard<ts::bravo::shared_mutex> lock(mutex);
+
+    std::thread t{[](ts::bravo::shared_mutex &mutex) {
+                    ts::bravo::Token token{0};
+                    CHECK(mutex.try_lock_shared(token) == false);
+                    CHECK(token == 0);
+                  },
+                  std::ref(mutex)};
+
+    t.join();
+  }
+
+  SECTION("writer-writer")
+  {
+    ts::bravo::shared_mutex mutex;
+    std::lock_guard<ts::bravo::shared_mutex> lock(mutex);
+
+    std::thread t{[](ts::bravo::shared_mutex &mutex) { CHECK(mutex.try_lock() 
== false); }, std::ref(mutex)};
+
+    t.join();
+  }
+}
+
+TEST_CASE("BRAVO - multiple try-lock", "[libts][BRAVO]")
+{
+  SECTION("rwrw")
+  {
+    ts::bravo::shared_mutex mutex;
+    int i = 0;
+
+    {
+      ts::bravo::Token token{0};
+      CHECK(mutex.try_lock_shared(token));
+      CHECK(i == 0);
+      mutex.unlock_shared(token);
+    }
+
+    {
+      CHECK(mutex.try_lock());
+      CHECK(++i == 1);
+      mutex.unlock();
+    }
+
+    {
+      ts::bravo::Token token{0};
+      CHECK(mutex.try_lock_shared(token));
+      CHECK(i == 1);
+      mutex.unlock_shared(token);
+    }
+
+    {
+      CHECK(mutex.try_lock());
+      CHECK(++i == 2);
+      mutex.unlock();
+    }
+
+    CHECK(i == 2);
+  }
+}
+
+TEST_CASE("BRAVO - check with race", "[libts][BRAVO]")
+{
+  SECTION("reader-reader")
+  {
+    ts::bravo::shared_mutex mutex;
+    int i = 0;
+
+    std::thread t1{[&](ts::bravo::shared_mutex &mutex) {
+                     ts::bravo::shared_lock<ts::bravo::shared_mutex> 
lock(mutex);
+                     CHECK(lock.owns_lock() == true);
+                     CHECK(i == 0);
+                   },
+                   std::ref(mutex)};
+
+    std::thread t2{[&](ts::bravo::shared_mutex &mutex) {
+                     ts::bravo::shared_lock<ts::bravo::shared_mutex> 
lock(mutex);
+                     CHECK(lock.owns_lock() == true);
+                     CHECK(i == 0);
+                   },
+                   std::ref(mutex)};
+
+    t1.join();
+    t2.join();
+
+    CHECK(i == 0);
+  }
+
+  SECTION("reader-writer")
+  {
+    ts::bravo::shared_mutex mutex;
+    int i = 0;
+
+    std::thread t1{[&](ts::bravo::shared_mutex &mutex) {
+                     ts::bravo::shared_lock<ts::bravo::shared_mutex> 
lock(mutex);
+                     CHECK(lock.owns_lock() == true);
+                     CHECK(i == 0);
+                     std::this_thread::sleep_for(100ms);
+                   },
+                   std::ref(mutex)};
+
+    std::thread t2{[&](ts::bravo::shared_mutex &mutex) {
+                     std::this_thread::sleep_for(50ms);
+                     std::lock_guard<ts::bravo::shared_mutex> lock(mutex);
+                     CHECK(++i == 1);
+                   },
+                   std::ref(mutex)};
+
+    t1.join();
+    t2.join();
+
+    CHECK(i == 1);
+  }
+
+  SECTION("writer-reader")
+  {
+    ts::bravo::shared_mutex mutex;
+    int i = 0;
+
+    std::thread t1{[&](ts::bravo::shared_mutex &mutex) {
+                     std::lock_guard<ts::bravo::shared_mutex> lock(mutex);
+                     std::this_thread::sleep_for(100ms);
+                     CHECK(++i == 1);
+                   },
+                   std::ref(mutex)};
+
+    std::thread t2{[&](ts::bravo::shared_mutex &mutex) {
+                     std::this_thread::sleep_for(50ms);
+                     ts::bravo::shared_lock<ts::bravo::shared_mutex> 
lock(mutex);
+                     CHECK(lock.owns_lock() == true);
+                     CHECK(i == 1);
+                   },
+                   std::ref(mutex)};
+
+    t1.join();
+    t2.join();
+
+    CHECK(i == 1);
+  }
+
+  SECTION("writer-writer")
+  {
+    ts::bravo::shared_mutex mutex;
+    int i = 0;
+
+    std::thread t1{[&](ts::bravo::shared_mutex &mutex) {
+                     std::lock_guard<ts::bravo::shared_mutex> lock(mutex);
+                     std::this_thread::sleep_for(100ms);
+                     CHECK(++i == 1);
+                   },
+                   std::ref(mutex)};
+
+    std::thread t2{[&](ts::bravo::shared_mutex &mutex) {
+                     std::this_thread::sleep_for(50ms);
+                     std::lock_guard<ts::bravo::shared_mutex> lock(mutex);
+                     CHECK(++i == 2);
+                   },
+                   std::ref(mutex)};
+
+    t1.join();
+    t2.join();
+
+    CHECK(i == 2);
+  }
+}
diff --git a/src/tscpp/util/Makefile.am b/src/tscpp/util/Makefile.am
index fa51549fbc..5845a2e41c 100644
--- a/src/tscpp/util/Makefile.am
+++ b/src/tscpp/util/Makefile.am
@@ -36,7 +36,9 @@ test_tscpputil_CPPFLAGS = $(AM_CPPFLAGS)\
        -I$(abs_top_srcdir)/tests/include @SWOC_INCLUDES@
 
 test_tscpputil_CXXFLAGS = -Wno-array-bounds $(AM_CXXFLAGS)
-test_tscpputil_LDADD = libtscpputil.la @SWOC_LIBS@
+test_tscpputil_LDADD = \
+       libtscpputil.la \
+       @SWOC_LIBS@
 test_tscpputil_SOURCES = \
        unit_tests/unit_test_main.cc \
        unit_tests/test_LocalBuffer.cc \

Reply via email to