You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@trafficserver.apache.org by "masaori335 (via GitHub)" <gi...@apache.org> on 2023/02/09 06:31:15 UTC

[GitHub] [trafficserver] masaori335 opened a new pull request, #9394: Add BRAVO Reader-Writer Lock

masaori335 opened a new pull request, #9394:
URL: https://github.com/apache/trafficserver/pull/9394

   # Abstract
   
   Introduce `ts::bravo::shared_mutex` as an alternative to the `std::shared_mutex` or `pthread_rwlock_t`.
   
   As we know, `std::shared_mutex` or `pthread_rwlock_t` (almost equivalent on Linux) doesn't scale on a multi-thread application like ATS. BRAVO is a reader-writer lock algorithm published by Dave Dice and Alex Kogan at USENIX 2019[*1]. The algorithm acts as an accelerator layer for the reader lock. Please note that this doesn't accelerate the writer lock, but almost no penalty. This algorithm is useful for read-heavy use cases. For the details of the algorithm, please check the paper.
   
   # Implementation
   
   The puzpuzpuz/xsync's `RBMutex` is an algorithm implementation in go-lang. `ts::bravo::shared_mutex` followed it in C++.
   
   ## Exclusive locking (writer)
   
   `ts::bravo::shared_mutex` can be a drop-in replacement.
   ```
   ts::bravo::shared_mutex mutex; 
   std::lock_guard lock(mutex);
   ```
   
   ## Shared locking (reader)
   
   To handle `ts::bravo::Token`, it needs `ts::bravo::shared_lock`, but you can use it like `std::shared_lock`.
   ```
   ts::bravo::shared_mutex mutex; 
   ts::bravo::shared_lock lock(mutex);
   ```
   
   # Micro Benchmark
   
   `benchmark_shared_mutex.cc` is Catch2 based micro benchmark tool. You can adjust threads and the rate of reading and writing.
   
   ```
   $ taskset -c 0-63 ./benchmark_shared_mutex --ts-nthreads 64 --ts-nloop 1000 --ts-nread 100 --ts-nwrite 1
   ```
   
   ## read: 1000, write: 0
   
     | ts::shared_mutex | ts::bravo::shared_mutex
   -- | -- | --
   1 | 4.36742 | 1.09271
   2 | 76.5058 | 1.08926
   4 | 126.713 | 1.11642
   8 | 232.651 | 1.18535
   16 | 498.872 | 1.80362
   32 | 1012.19 | 2.17977
   64 | 2172.4 | 5.7694
   <img width="535" alt="Screenshot 2023-02-09 at 15 20 18" src="https://user-images.githubusercontent.com/741391/217733963-77c9d910-b303-4387-aa6d-192eaae24dca.png">
   
   ## read: 1000, write: 1
   
     | ts::shared_mutex | ts::bravo::shared_mutex
   -- | -- | --
   1 | 4.3714 | 3.19028
   2 | 59.3886 | 17.5586
   4 | 132.736 | 53.6457
   8 | 249.857 | 112.068
   16 | 586.322 | 128.144
   32 | 1375.09 | 227.752
   64 | 5240.33 | 1301.71
   
   <img width="535" alt="Screenshot 2023-02-09 at 15 20 27" src="https://user-images.githubusercontent.com/741391/217733992-a03d087b-0c88-4c6b-8e60-ab3b03373cdd.png">
   
   # References
   
   [*1] 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
   
   [*2] xsync - Concurrent data structures for Go
    https://github.com/puzpuzpuz/xsync


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ezelkow1 commented on pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ezelkow1 (via GitHub)" <gi...@apache.org>.
ezelkow1 commented on PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#issuecomment-1554938042

   [approve ci centos]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1111109669


##########
src/tscpp/util/unit_tests/benchmark_shared_mutex.cc:
##########
@@ -0,0 +1,130 @@
+/** @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;
+thread_local int counter = 0;
+
+template <typename T, typename S>
+void
+run(T &mutex)
+{
+  std::thread list[conf.nthreads];
+
+  for (int i = 0; i < conf.nthreads; i++) {
+    new (&list[i]) std::thread{[](T &mutex) {

Review Comment:
   Why not just:
   ```
   list[i] = std::thread{...
   ```
   ?  std::thread isn't copyable, but it's moveable.  What you're doing assumes that ~std::thread() is vacuous when the thread is constructed with the default constructor.  That's probably true in this case, but it's better practice to not make such assumptions unless it's really necessary.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1111104582


##########
include/tscpp/util/Bravo.h:
##########
@@ -0,0 +1,382 @@
+/** @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 <array>
+#include <atomic>
+#include <cassert>
+#include <chrono>
+#include <shared_mutex>
+#include <thread>
+
+namespace ts::bravo
+{
+static inline uint32_t
+mix32(uint64_t z)
+{
+  z = (z ^ (z >> 33)) * 0xff51afd7ed558ccdL;
+  z = (z ^ (z >> 33)) * 0xc4ceb9fe1a85ec53L;
+  return static_cast<uint32_t>(z >> 32);
+}
+
+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

Review Comment:
   Why can't bravo shared mutexes just std::shared_lock?  If they require a custom version of shared_lock, can they use std::unique_lock?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] lzx404243 commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "lzx404243 (via GitHub)" <gi...@apache.org>.
lzx404243 commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1143603357


##########
include/tscpp/util/Bravo.h:
##########
@@ -0,0 +1,375 @@
+/** @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 <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() = default;
+  shared_lock(Mutex &m) : _mutex(&m) { lock(); }
+
+  ~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)
+  {
+    // Fast path
+    if (_mutex.read_bias.load(std::memory_order_acquire)) {
+      size_t index = DenseThreadId::self();
+      for (size_t i = 0; i < SLOT_SIZE; ++i) {

Review Comment:
   Not sure why there is a for loop in the locking path, although it can be rare for it to iterate more than once as it stops once there is no collision. Not sure how well this performs compared to just falling back to the slow path upon a collision, as in the BRAVO paper. The latter seems to have more predictable performance characteristics, as each thread can only go to one slot in the table(versus every thread can go to any slots which can increase future chance of collision).
   
   btw, in your current implementation with the use of `DenseThreadId`, will there be any collision at all?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] masaori335 commented on pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "masaori335 (via GitHub)" <gi...@apache.org>.
masaori335 commented on PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#issuecomment-1467041666

   [approve ci autest]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] masaori335 merged pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "masaori335 (via GitHub)" <gi...@apache.org>.
masaori335 merged PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] lzx404243 commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "lzx404243 (via GitHub)" <gi...@apache.org>.
lzx404243 commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1143603357


##########
include/tscpp/util/Bravo.h:
##########
@@ -0,0 +1,375 @@
+/** @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 <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() = default;
+  shared_lock(Mutex &m) : _mutex(&m) { lock(); }
+
+  ~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)
+  {
+    // Fast path
+    if (_mutex.read_bias.load(std::memory_order_acquire)) {
+      size_t index = DenseThreadId::self();
+      for (size_t i = 0; i < SLOT_SIZE; ++i) {

Review Comment:
   Not sure why there is a for loop in the locking path, although it can be rare for it to iterate more than once as it stops once there is no collision. Not sure how much better this is compared to just falling back to the slow path upon a collision, as in the BRAVO paper. The latter seems to have more predictable performance characteristics, as each thread can only go to one slot in the table(versus every thread can go to any slots which can increase future chance of collision).
   
   btw, in your current implementation with the use of `DenseThreadId`, will there be any collision at all?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] masaori335 commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "masaori335 (via GitHub)" <gi...@apache.org>.
masaori335 commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1148716306


##########
include/tscpp/util/Bravo.h:
##########
@@ -0,0 +1,375 @@
+/** @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 <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() = default;
+  shared_lock(Mutex &m) : _mutex(&m) { lock(); }
+
+  ~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)
+  {
+    // Fast path
+    if (_mutex.read_bias.load(std::memory_order_acquire)) {
+      size_t index = DenseThreadId::self();
+      for (size_t i = 0; i < SLOT_SIZE; ++i) {

Review Comment:
   My understanding is, as far as the `SLOT_SIZE` is larger than `DenthThreadId::_num_possible_values`, there should be no corrosion. Probably, we can add static asserts to make sure it and get rid of this for loop.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1111104582


##########
include/tscpp/util/Bravo.h:
##########
@@ -0,0 +1,382 @@
+/** @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 <array>
+#include <atomic>
+#include <cassert>
+#include <chrono>
+#include <shared_mutex>
+#include <thread>
+
+namespace ts::bravo
+{
+static inline uint32_t
+mix32(uint64_t z)
+{
+  z = (z ^ (z >> 33)) * 0xff51afd7ed558ccdL;
+  z = (z ^ (z >> 33)) * 0xc4ceb9fe1a85ec53L;
+  return static_cast<uint32_t>(z >> 32);
+}
+
+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

Review Comment:
   Why can't bravo shared mutexes just std::shared_lock?  If they require a custom version of shared_lock, can they use std::unique_lock?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] lzx404243 commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "lzx404243 (via GitHub)" <gi...@apache.org>.
lzx404243 commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1143579691


##########
include/tscpp/util/Bravo.h:
##########
@@ -0,0 +1,375 @@
+/** @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 <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() = default;
+  shared_lock(Mutex &m) : _mutex(&m) { lock(); }
+
+  ~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)
+  {
+    // Fast path
+    if (_mutex.read_bias.load(std::memory_order_acquire)) {
+      size_t index = DenseThreadId::self();
+      for (size_t i = 0; i < SLOT_SIZE; ++i) {
+        index       = (index + i) % 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)
+  {
+    // Fast path
+    if (_mutex.read_bias.load(std::memory_order_acquire)) {
+      size_t index = DenseThreadId::self();
+      for (size_t i = 0; i < SLOT_SIZE; ++i) {
+        index       = (index + i) % 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 = {};

Review Comment:
   It seems unlike the original BRAVO where a global reader table is used, your implementation has a reader table per lock instance. This will increase the per-lock memory footprint, with potential performance benefit. May benchmark the memory usage as well to see whether the trade-off is worthwhile.



##########
include/tscpp/util/Bravo.h:
##########
@@ -0,0 +1,375 @@
+/** @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 <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() = default;
+  shared_lock(Mutex &m) : _mutex(&m) { lock(); }
+
+  ~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)
+  {
+    // Fast path
+    if (_mutex.read_bias.load(std::memory_order_acquire)) {
+      size_t index = DenseThreadId::self();
+      for (size_t i = 0; i < SLOT_SIZE; ++i) {

Review Comment:
   Not sure why there is a for loop in the locking path, although it can be rare for it to iterate more than once as it stops once there is no collision. Not sure how much better this is compared to just falling back to the slow path upon a collision. The latter seems to have more predictable performance characteristics, as each thread can go to one slot in the table(versus every thread can go to any slots which can increase future chance of collision).
   
   btw, in your current implementation with the use of `DenseThreadId`, will there be any collision at all?



##########
include/tscpp/util/Bravo.h:
##########
@@ -0,0 +1,375 @@
+/** @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 <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() = default;
+  shared_lock(Mutex &m) : _mutex(&m) { lock(); }

Review Comment:
   might also want the other constructors:
   ```
    shared_lock(Mutex &m, std::try_to_lock_t) : _mutex(&m) { try_lock(); }
    shared_lock(Mutex &m, std::defer_lock_t) noexcept : _mutex(&m) {}
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1111109669


##########
src/tscpp/util/unit_tests/benchmark_shared_mutex.cc:
##########
@@ -0,0 +1,130 @@
+/** @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;
+thread_local int counter = 0;
+
+template <typename T, typename S>
+void
+run(T &mutex)
+{
+  std::thread list[conf.nthreads];
+
+  for (int i = 0; i < conf.nthreads; i++) {
+    new (&list[i]) std::thread{[](T &mutex) {

Review Comment:
   Why not just:
   ```
   list[i] = std::thread{...
   ```
   ?  std::thread isn't copyable, but it's moveable.  What your doing assumes that ~std::thread() is vacuous when the thread is constructed with the default constructor.  That's probably true in this case, but it's better practice to not make such assumptions unless it's really necessary.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] masaori335 commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "masaori335 (via GitHub)" <gi...@apache.org>.
masaori335 commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1112478296


##########
include/tscpp/util/Bravo.h:
##########
@@ -0,0 +1,382 @@
+/** @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 <array>
+#include <atomic>
+#include <cassert>
+#include <chrono>
+#include <shared_mutex>
+#include <thread>
+
+namespace ts::bravo
+{
+static inline uint32_t
+mix32(uint64_t z)
+{
+  z = (z ^ (z >> 33)) * 0xff51afd7ed558ccdL;
+  z = (z ^ (z >> 33)) * 0xc4ceb9fe1a85ec53L;
+  return static_cast<uint32_t>(z >> 32);
+}
+
+using time_point = std::chrono::time_point<std::chrono::system_clock>;
+
+#ifdef __cpp_lib_hardware_interference_size
+using std::hardware_constructive_interference_size;

Review Comment:
   This is C++17 but only available with GCC 😿
   https://en.cppreference.com/w/cpp/thread/hardware_destructive_interference_size
   <img width="796" alt="Screenshot 2023-02-20 at 8 05 17" src="https://user-images.githubusercontent.com/741391/220233589-d0bbe5a8-5ed4-42d9-ae30-3086276c48d9.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1113677441


##########
include/tscpp/util/Bravo.h:
##########
@@ -0,0 +1,386 @@
+/** @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 <array>
+#include <atomic>
+#include <cassert>
+#include <chrono>
+#include <shared_mutex>
+#include <thread>
+
+namespace ts::bravo
+{
+static inline uint32_t
+mix32(uint64_t z)
+{
+  z = (z ^ (z >> 33)) * 0xff51afd7ed558ccdL;
+  z = (z ^ (z >> 33)) * 0xc4ceb9fe1a85ec53L;
+  return static_cast<uint32_t>(z >> 32);
+}
+
+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() = default;
+  shared_lock(Mutex &m) : _mutex(&m) { lock(); }
+
+  ~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.
+
+   SLOT_SIZE needs to be larget than number of threads for lock_shared to go fast-path.
+ */
+template <typename T = std::shared_mutex, size_t SLOT_SIZE = 4096, 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)

Review Comment:
   Why can't  lock_shared() just return the token.  Probably not faster after optimization, but seems clearer to me.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] masaori335 commented on pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "masaori335 (via GitHub)" <gi...@apache.org>.
masaori335 commented on PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#issuecomment-1502386898

   [approve ci cmake]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] lzx404243 commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "lzx404243 (via GitHub)" <gi...@apache.org>.
lzx404243 commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1143579691


##########
include/tscpp/util/Bravo.h:
##########
@@ -0,0 +1,375 @@
+/** @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 <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() = default;
+  shared_lock(Mutex &m) : _mutex(&m) { lock(); }
+
+  ~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)
+  {
+    // Fast path
+    if (_mutex.read_bias.load(std::memory_order_acquire)) {
+      size_t index = DenseThreadId::self();
+      for (size_t i = 0; i < SLOT_SIZE; ++i) {
+        index       = (index + i) % 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)
+  {
+    // Fast path
+    if (_mutex.read_bias.load(std::memory_order_acquire)) {
+      size_t index = DenseThreadId::self();
+      for (size_t i = 0; i < SLOT_SIZE; ++i) {
+        index       = (index + i) % 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 = {};

Review Comment:
   It seems unlike the original BRAVO where a global reader table is used, your implementation has a reader table per lock instance. This will increase the per-lock memory footprint, with potential performance benefit. May benchmark the memory usage as well to see whether the trade-off is worthwhile/acceptable.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#issuecomment-1558112162

   Masaori and I don' t agree as to whether this needs a longer unit test, so I'm bowing out and leaving it to other reviewers to break the tie.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] lzx404243 commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "lzx404243 (via GitHub)" <gi...@apache.org>.
lzx404243 commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1143603357


##########
include/tscpp/util/Bravo.h:
##########
@@ -0,0 +1,375 @@
+/** @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 <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() = default;
+  shared_lock(Mutex &m) : _mutex(&m) { lock(); }
+
+  ~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)
+  {
+    // Fast path
+    if (_mutex.read_bias.load(std::memory_order_acquire)) {
+      size_t index = DenseThreadId::self();
+      for (size_t i = 0; i < SLOT_SIZE; ++i) {

Review Comment:
   Not sure why there is a for loop in the locking path, although it can be rare for it to iterate more than once as it stops once there is no collision. Not sure how much better this is compared to just falling back to the slow path upon a collision, as in the BRaVO paper. The latter seems to have more predictable performance characteristics, as each thread can go to one slot in the table(versus every thread can go to any slots which can increase future chance of collision).
   
   btw, in your current implementation with the use of `DenseThreadId`, will there be any collision at all?



##########
include/tscpp/util/Bravo.h:
##########
@@ -0,0 +1,375 @@
+/** @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 <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() = default;
+  shared_lock(Mutex &m) : _mutex(&m) { lock(); }
+
+  ~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)
+  {
+    // Fast path
+    if (_mutex.read_bias.load(std::memory_order_acquire)) {
+      size_t index = DenseThreadId::self();
+      for (size_t i = 0; i < SLOT_SIZE; ++i) {

Review Comment:
   Not sure why there is a for loop in the locking path, although it can be rare for it to iterate more than once as it stops once there is no collision. Not sure how much better this is compared to just falling back to the slow path upon a collision, as in the BRAVO paper. The latter seems to have more predictable performance characteristics, as each thread can go to one slot in the table(versus every thread can go to any slots which can increase future chance of collision).
   
   btw, in your current implementation with the use of `DenseThreadId`, will there be any collision at all?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] masaori335 commented on pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "masaori335 (via GitHub)" <gi...@apache.org>.
masaori335 commented on PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#issuecomment-1447683208

   For the logic, having more complicated tests is better, but current tests have enough coverage to start testing this reader-writer lock widely. Let's have more eyeballs.
   
   For the benchmark, I noticed Catch2 Benchmark provides an option[*1] (`return` from the function) for the optimization and benchmark issue. I followed this way. Also, I checked with `objdump -d` and ran the benchmark on the lldb with "disassembly mode"[*2]. It has some instructions.
   
   [*1] https://github.com/catchorg/Catch2/blob/v2.13.8/docs/benchmarks.md#the-optimizer
   [*2] https://github.com/llvm/llvm-project/blob/main/lldb/examples/python/disassembly_mode.py
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] masaori335 commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "masaori335 (via GitHub)" <gi...@apache.org>.
masaori335 commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1148712545


##########
include/tscpp/util/Bravo.h:
##########
@@ -0,0 +1,375 @@
+/** @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 <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() = default;
+  shared_lock(Mutex &m) : _mutex(&m) { lock(); }
+
+  ~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)
+  {
+    // Fast path
+    if (_mutex.read_bias.load(std::memory_order_acquire)) {
+      size_t index = DenseThreadId::self();
+      for (size_t i = 0; i < SLOT_SIZE; ++i) {
+        index       = (index + i) % 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)
+  {
+    // Fast path
+    if (_mutex.read_bias.load(std::memory_order_acquire)) {
+      size_t index = DenseThreadId::self();
+      for (size_t i = 0; i < SLOT_SIZE; ++i) {
+        index       = (index + i) % 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 = {};

Review Comment:
   Right, I followed the approach of xsync's RBMutex. 
   
   https://github.com/puzpuzpuz/xsync/blob/main/rbmutex.go



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1111110852


##########
src/tscpp/util/unit_tests/benchmark_shared_mutex.cc:
##########
@@ -0,0 +1,130 @@
+/** @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;
+thread_local int counter = 0;
+
+template <typename T, typename S>
+void
+run(T &mutex)
+{
+  std::thread list[conf.nthreads];
+
+  for (int i = 0; i < conf.nthreads; i++) {
+    new (&list[i]) std::thread{[](T &mutex) {

Review Comment:
   It seems like the only advantage of using a lambda here is that you avoid having to pick a name for the thread function.  I think it would be more readable just using a conventional, named function for the thread function.
   
   On the other hand, @SolidWallOfCode smiles and sighs with contentment whenever anyone uses a lambda, so there is that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#issuecomment-1439240768

   Here is a unit test I wrote in a previous effort to build a better shared_mutex:  https://github.com/ywkaras/trafficserver/blob/OWMR/src/tscore/unit_tests/test_OneWriterMultiReader.cc .


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#issuecomment-1425066466

   Perhaps compare also to https://github.com/apache/trafficserver/pull/9154 .


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#issuecomment-1435775317

   ts::scalable_shared_mutex uses the class DenseThreadId ( https://github.com/apache/trafficserver/pull/9154/files#diff-5d4184d5004b8c73554cb88f0bfe7f0c4711c7394c5a15e24daaaff8962d1076R177 ).  It provides a thread ID for currently running threads, that's in the range from 0 to one less than the number of currently-running threads.  A new thread may reuse the ID of an exited thread.  (This could be a performance issue with dynamic threading, but ATS seems to do little of that.)  So `dense thread ID % num slots` would then be a reasonable hash function, with perhaps fewer collisions.  ts::scalable_shared_mutex always assume there are at least as many slots as currently running threads.  This avoids complex logic to deal with collisions.  I notice that you hard-coded 4096 as the number of slots.  I suspect there are few if any ATS proxies in prod that run optimally with more than 4096 simultaneously active threads.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#issuecomment-1435772499

   With my ts::scalable_shared_mutex, with 16 threads, it had worse performance that ts::shared_mutex.  So maybe check the performance of this mutex with 16 threads?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#issuecomment-1440324440

   It looks like there are still a few areas that need improvement.  But, I think the next step is to agree on what testing is needed for verification of proper, logically correct locking.  If you think your current logic testing is adequate, I think we should get at least one more reviewer, to break the tie.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] masaori335 commented on pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "masaori335 (via GitHub)" <gi...@apache.org>.
masaori335 commented on PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#issuecomment-1437724766

   > With my ts::scalable_shared_mutex, with 16 threads, it had worse performance that ts::shared_mutex. So maybe check the performance of this mutex with 16 threads?
   
   Please look at the table and graph above. The benchmark is measured with 1,2,4,8,16,32, and 64 threads. ts::brovo::shared_mutex shows better performance with fewer threads.
   
   > ts::scalable_shared_mutex uses the class DenseThreadId...
   
   IIUC, reducing collision between threads by using the hash function (mix32) is one of the key ideas of BRAVO. 
   However, for ATS, we might be able to reduce the slots with your DenseThreadId, because we don't use dynamic threading. I'll try to use it and measure performance.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1112470722


##########
src/tscpp/util/unit_tests/test_Bravo.cc:
##########
@@ -0,0 +1,192 @@
+/** @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 with try-lock", "[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 - 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);
+                     ++i;
+                   },
+                   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);
+                     ++i;
+                   },
+                   std::ref(mutex)};
+
+    t1.join();
+    t2.join();
+
+    CHECK(i == 2);

Review Comment:
   Hmm why take a shared lock to write?  Seems like `i` could end up being 1 not 2.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1111116672


##########
include/tscpp/util/Bravo.h:
##########
@@ -0,0 +1,382 @@
+/** @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 <array>
+#include <atomic>
+#include <cassert>
+#include <chrono>
+#include <shared_mutex>
+#include <thread>
+
+namespace ts::bravo
+{
+static inline uint32_t
+mix32(uint64_t z)
+{
+  z = (z ^ (z >> 33)) * 0xff51afd7ed558ccdL;
+  z = (z ^ (z >> 33)) * 0xc4ceb9fe1a85ec53L;
+  return static_cast<uint32_t>(z >> 32);
+}
+
+using time_point = std::chrono::time_point<std::chrono::system_clock>;
+
+#ifdef __cpp_lib_hardware_interference_size
+using std::hardware_constructive_interference_size;

Review Comment:
   It seems like this constant is only available in newer versions of gcc and clang.  We could maybe get the cache line size out of /proc/cpuinfo when the target is Linux?
   ```
   wkaras ~
   $ grep -e cache_alignment -e processor /proc/cpuinfo | head -n 4
   processor	: 0
   cache_alignment	: 64
   processor	: 1
   cache_alignment	: 64
   wkaras ~
   $
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#issuecomment-1439220094

   Would be good if Apple people looked at this.  Maybe @vmamidi or @cmcfarlen .  Apple was talking about having thread_local HostDBs.  Maybe if HostDB had a faster shared_mutex, that complexity could be avoided.  Yahoo has deprioritized exploration of the use of proxy hosts with many cores, so I won't get many brownie points for giving this the attention that it deserves.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] masaori335 commented on pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "masaori335 (via GitHub)" <gi...@apache.org>.
masaori335 commented on PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#issuecomment-1439523746

   Microbenchmark with DenseThreadId is below. We can go with it because it has no collision and we can reduce the slot size.
   
   ## read: 1000, write: 0
   
      |ts::bravo::shared_mutex (DenseThreadId)
   -- |--     
   1  |1.02784
   2  |1.04234
   4  |1.06245
   8  |1.14418
   16 |1.33218
   32 |2.40103
   64 |3.26439
   
   ## read: 1000, write: 1
   
      |ts::bravo::shared_mutex (DenseThreadId)
   -- |--
   1  |1.25493
   2  |47.5607
   4  |85.9465
   8  |130.392
   16 |106.654
   32 |237.198
   64 |1259.23
   
   <img width="1095" alt="Screenshot 2023-02-22 at 15 47 28" src="https://user-images.githubusercontent.com/741391/220544166-c90411ba-1079-4517-905c-3298e969d429.png">
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1111103649


##########
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

Review Comment:
   I'd suggest changing this to:
   ```
   Copyright (c) 2021 Andrey Pechkurov (MIT License)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] masaori335 commented on a diff in pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "masaori335 (via GitHub)" <gi...@apache.org>.
masaori335 commented on code in PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#discussion_r1112474964


##########
src/tscpp/util/unit_tests/test_Bravo.cc:
##########
@@ -0,0 +1,192 @@
+/** @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 with try-lock", "[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 - 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);
+                     ++i;
+                   },
+                   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);
+                     ++i;
+                   },
+                   std::ref(mutex)};
+
+    t1.join();
+    t2.join();
+
+    CHECK(i == 2);

Review Comment:
   I'll get rid of this. Initially, incrementing `i` to prevent optimization and do nothing. But now it has some checks. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] ywkaras commented on pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "ywkaras (via GitHub)" <gi...@apache.org>.
ywkaras commented on PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#issuecomment-1445501804

   A common problem with performance testing is that, since the code is executing to time it only, not to use any results, the optimizer will remove some or all of the code you're actually trying to time.  One way to double check is to use 'objdump -d' to disassemble, check if there's clearly too few assembly instructions corresponding to the source code.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [trafficserver] lzx404243 commented on pull request #9394: Add BRAVO Reader-Writer Lock

Posted by "lzx404243 (via GitHub)" <gi...@apache.org>.
lzx404243 commented on PR #9394:
URL: https://github.com/apache/trafficserver/pull/9394#issuecomment-1476683740

   I created a test build replacing all RW locks(`std::shared_mutex`, `ts::shared_mutex`, `ink_rwlock`) with `ts::bravo::shared_mutex` and ran it on a prod box. See no performance degradation nor crashes.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org