folly: build with -Wunused-parameter
[folly.git] / folly / SharedMutex.h
1 /*
2  * Copyright 2015 Facebook, Inc.
3  *
4  * Licensed under the Apache License, Version 2.0 (the "License");
5  * you may not use this file except in compliance with the License.
6  * You may obtain a copy of the License at
7  *
8  *   http://www.apache.org/licenses/LICENSE-2.0
9  *
10  * Unless required by applicable law or agreed to in writing, software
11  * distributed under the License is distributed on an "AS IS" BASIS,
12  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13  * See the License for the specific language governing permissions and
14  * limitations under the License.
15  */
16
17 // @author Nathan Bronson (ngbronson@fb.com)
18
19 #pragma once
20
21 #include <stdint.h>
22 #include <atomic>
23 #include <thread>
24 #include <type_traits>
25 #include <folly/Likely.h>
26 #include <folly/detail/CacheLocality.h>
27 #include <folly/detail/Futex.h>
28 #include <sys/resource.h>
29
30 // SharedMutex is a reader-writer lock.  It is small, very fast, scalable
31 // on multi-core, and suitable for use when readers or writers may block.
32 // Unlike most other reader-writer locks, its throughput with concurrent
33 // readers scales linearly; it is able to acquire and release the lock
34 // in shared mode without cache line ping-ponging.  It is suitable for
35 // a wide range of lock hold times because it starts with spinning,
36 // proceeds to using sched_yield with a preemption heuristic, and then
37 // waits using futex and precise wakeups.
38 //
39 // SharedMutex provides all of the methods of folly::RWSpinLock,
40 // boost::shared_mutex, boost::upgrade_mutex, and C++14's
41 // std::shared_timed_mutex.  All operations that can block are available
42 // in try, try-for, and try-until (system_clock or steady_clock) versions.
43 //
44 // SharedMutexReadPriority gives priority to readers,
45 // SharedMutexWritePriority gives priority to writers.  SharedMutex is an
46 // alias for SharedMutexWritePriority, because writer starvation is more
47 // likely than reader starvation for the read-heavy workloads targetted
48 // by SharedMutex.
49 //
50 // In my tests SharedMutex is as good or better than the other
51 // reader-writer locks in use at Facebook for almost all use cases,
52 // sometimes by a wide margin.  (If it is rare that there are actually
53 // concurrent readers then RWSpinLock can be a few nanoseconds faster.)
54 // I compared it to folly::RWSpinLock, folly::RWTicketSpinLock64,
55 // boost::shared_mutex, pthread_rwlock_t, and a RWLock that internally uses
56 // spinlocks to guard state and pthread_mutex_t+pthread_cond_t to block.
57 // (Thrift's ReadWriteMutex is based underneath on pthread_rwlock_t.)
58 // It is generally as good or better than the rest when evaluating size,
59 // speed, scalability, or latency outliers.  In the corner cases where
60 // it is not the fastest (such as single-threaded use or heavy write
61 // contention) it is never very much worse than the best.  See the bottom
62 // of folly/test/SharedMutexTest.cpp for lots of microbenchmark results.
63 //
64 // Comparison to folly::RWSpinLock:
65 //
66 //  * SharedMutex is faster than RWSpinLock when there are actually
67 //    concurrent read accesses (sometimes much faster), and ~5 nanoseconds
68 //    slower when there is not actually any contention.  SharedMutex is
69 //    faster in every (benchmarked) scenario where the shared mode of
70 //    the lock is actually useful.
71 //
72 //  * Concurrent shared access to SharedMutex scales linearly, while total
73 //    RWSpinLock throughput drops as more threads try to access the lock
74 //    in shared mode.  Under very heavy read contention SharedMutex can
75 //    be two orders of magnitude faster than RWSpinLock (or any reader
76 //    writer lock that doesn't use striping or deferral).
77 //
78 //  * SharedMutex can safely protect blocking calls, because after an
79 //    initial period of spinning it waits using futex().
80 //
81 //  * RWSpinLock prioritizes readers, SharedMutex has both reader- and
82 //    writer-priority variants, but defaults to write priority.
83 //
84 //  * RWSpinLock's upgradeable mode blocks new readers, while SharedMutex's
85 //    doesn't.  Both semantics are reasonable.  The boost documentation
86 //    doesn't explicitly talk about this behavior (except by omitting
87 //    any statement that those lock modes conflict), but the boost
88 //    implementations do allow new readers while the upgradeable mode
89 //    is held.  See https://github.com/boostorg/thread/blob/master/
90 //      include/boost/thread/pthread/shared_mutex.hpp
91 //
92 //  * RWSpinLock::UpgradedHolder maps to SharedMutex::UpgradeHolder
93 //    (UpgradeableHolder would be even more pedantically correct).
94 //    SharedMutex's holders have fewer methods (no reset) and are less
95 //    tolerant (promotion and downgrade crash if the donor doesn't own
96 //    the lock, and you must use the default constructor rather than
97 //    passing a nullptr to the pointer constructor).
98 //
99 // Both SharedMutex and RWSpinLock provide "exclusive", "upgrade",
100 // and "shared" modes.  At all times num_threads_holding_exclusive +
101 // num_threads_holding_upgrade <= 1, and num_threads_holding_exclusive ==
102 // 0 || num_threads_holding_shared == 0.  RWSpinLock has the additional
103 // constraint that num_threads_holding_shared cannot increase while
104 // num_threads_holding_upgrade is non-zero.
105 //
106 // Comparison to the internal RWLock:
107 //
108 //  * SharedMutex doesn't allow a maximum reader count to be configured,
109 //    so it can't be used as a semaphore in the same way as RWLock.
110 //
111 //  * SharedMutex is 4 bytes, RWLock is 256.
112 //
113 //  * SharedMutex is as fast or faster than RWLock in all of my
114 //    microbenchmarks, and has positive rather than negative scalability.
115 //
116 //  * RWLock and SharedMutex are both writer priority locks.
117 //
118 //  * SharedMutex avoids latency outliers as well as RWLock.
119 //
120 //  * SharedMutex uses different names (t != 0 below):
121 //
122 //    RWLock::lock(0)    => SharedMutex::lock()
123 //
124 //    RWLock::lock(t)    => SharedMutex::try_lock_for(milliseconds(t))
125 //
126 //    RWLock::tryLock()  => SharedMutex::try_lock()
127 //
128 //    RWLock::unlock()   => SharedMutex::unlock()
129 //
130 //    RWLock::enter(0)   => SharedMutex::lock_shared()
131 //
132 //    RWLock::enter(t)   =>
133 //        SharedMutex::try_lock_shared_for(milliseconds(t))
134 //
135 //    RWLock::tryEnter() => SharedMutex::try_lock_shared()
136 //
137 //    RWLock::leave()    => SharedMutex::unlock_shared()
138 //
139 //  * RWLock allows the reader count to be adjusted by a value other
140 //    than 1 during enter() or leave(). SharedMutex doesn't currently
141 //    implement this feature.
142 //
143 //  * RWLock's methods are marked const, SharedMutex's aren't.
144 //
145 // Reader-writer locks have the potential to allow concurrent access
146 // to shared read-mostly data, but in practice they often provide no
147 // improvement over a mutex.  The problem is the cache coherence protocol
148 // of modern CPUs.  Coherence is provided by making sure that when a cache
149 // line is written it is present in only one core's cache.  Since a memory
150 // write is required to acquire a reader-writer lock in shared mode, the
151 // cache line holding the lock is invalidated in all of the other caches.
152 // This leads to cache misses when another thread wants to acquire or
153 // release the lock concurrently.  When the RWLock is colocated with the
154 // data it protects (common), cache misses can also continue occur when
155 // a thread that already holds the lock tries to read the protected data.
156 //
157 // Ideally, a reader-writer lock would allow multiple cores to acquire
158 // and release the lock in shared mode without incurring any cache misses.
159 // This requires that each core records its shared access in a cache line
160 // that isn't read or written by other read-locking cores.  (Writers will
161 // have to check all of the cache lines.)  Typical server hardware when
162 // this comment was written has 16 L1 caches and cache lines of 64 bytes,
163 // so a lock striped over all L1 caches would occupy a prohibitive 1024
164 // bytes.  Nothing says that we need a separate set of per-core memory
165 // locations for each lock, however.  Each SharedMutex instance is only
166 // 4 bytes, but all locks together share a 2K area in which they make a
167 // core-local record of lock acquisitions.
168 //
169 // SharedMutex's strategy of using a shared set of core-local stripes has
170 // a potential downside, because it means that acquisition of any lock in
171 // write mode can conflict with acquisition of any lock in shared mode.
172 // If a lock instance doesn't actually experience concurrency then this
173 // downside will outweight the upside of improved scalability for readers.
174 // To avoid this problem we dynamically detect concurrent accesses to
175 // SharedMutex, and don't start using the deferred mode unless we actually
176 // observe concurrency.  See kNumSharedToStartDeferring.
177 //
178 // It is explicitly allowed to call lock_unshared() from a different
179 // thread than lock_shared(), so long as they are properly paired.
180 // lock_unshared() needs to find the location at which lock_shared()
181 // recorded the lock, which might be in the lock itself or in any of
182 // the shared slots.  If you can conveniently pass state from lock
183 // acquisition to release then the fastest mechanism is to std::move
184 // the SharedMutex::ReadHolder instance or an SharedMutex::Token (using
185 // lock_shared(Token&) and unlock_sahred(Token&)).  The guard or token
186 // will tell unlock_shared where in deferredReaders[] to look for the
187 // deferred lock.  The Token-less version of unlock_shared() works in all
188 // cases, but is optimized for the common (no inter-thread handoff) case.
189 //
190 // In both read- and write-priority mode, a waiting lock() (exclusive mode)
191 // only blocks readers after it has waited for an active upgrade lock to be
192 // released; until the upgrade lock is released (or upgraded or downgraded)
193 // readers will still be able to enter.  Preferences about lock acquisition
194 // are not guaranteed to be enforced perfectly (even if they were, there
195 // is theoretically the chance that a thread could be arbitrarily suspended
196 // between calling lock() and SharedMutex code actually getting executed).
197 //
198 // try_*_for methods always try at least once, even if the duration
199 // is zero or negative.  The duration type must be compatible with
200 // std::chrono::steady_clock.  try_*_until methods also always try at
201 // least once.  std::chrono::system_clock and std::chrono::steady_clock
202 // are supported.
203 //
204 // If you have observed by profiling that your SharedMutex-s are getting
205 // cache misses on deferredReaders[] due to another SharedMutex user, then
206 // you can use the tag type plus the RWDEFERREDLOCK_DECLARE_STATIC_STORAGE
207 // macro to create your own instantiation of the type.  The contention
208 // threshold (see kNumSharedToStartDeferring) should make this unnecessary
209 // in all but the most extreme cases.  Make sure to check that the
210 // increased icache and dcache footprint of the tagged result is worth it.
211
212 namespace folly {
213
214 struct SharedMutexToken {
215   enum class Type : uint16_t {
216     INVALID = 0,
217     INLINE_SHARED,
218     DEFERRED_SHARED,
219   };
220
221   Type type_;
222   uint16_t slot_;
223 };
224
225 template <bool ReaderPriority,
226           typename Tag_ = void,
227           template <typename> class Atom = std::atomic,
228           bool BlockImmediately = false>
229 class SharedMutexImpl {
230  public:
231   static constexpr bool kReaderPriority = ReaderPriority;
232   typedef Tag_ Tag;
233
234   typedef SharedMutexToken Token;
235
236   class ReadHolder;
237   class UpgradeHolder;
238   class WriteHolder;
239
240   constexpr SharedMutexImpl() : state_(0) {}
241
242   SharedMutexImpl(const SharedMutexImpl&) = delete;
243   SharedMutexImpl(SharedMutexImpl&&) = delete;
244   SharedMutexImpl& operator = (const SharedMutexImpl&) = delete;
245   SharedMutexImpl& operator = (SharedMutexImpl&&) = delete;
246
247   // It is an error to destroy an SharedMutex that still has
248   // any outstanding locks.  This is checked if NDEBUG isn't defined.
249   // SharedMutex's exclusive mode can be safely used to guard the lock's
250   // own destruction.  If, for example, you acquire the lock in exclusive
251   // mode and then observe that the object containing the lock is no longer
252   // needed, you can unlock() and then immediately destroy the lock.
253   // See https://sourceware.org/bugzilla/show_bug.cgi?id=13690 for a
254   // description about why this property needs to be explicitly mentioned.
255   ~SharedMutexImpl() {
256     auto state = state_.load(std::memory_order_relaxed);
257     if (UNLIKELY((state & kHasS) != 0)) {
258       cleanupTokenlessSharedDeferred(state);
259     }
260
261 #ifndef NDEBUG
262     // if a futexWait fails to go to sleep because the value has been
263     // changed, we don't necessarily clean up the wait bits, so it is
264     // possible they will be set here in a correct system
265     assert((state & ~(kWaitingAny | kMayDefer)) == 0);
266     if ((state & kMayDefer) != 0) {
267       for (uint32_t slot = 0; slot < kMaxDeferredReaders; ++slot) {
268         auto slotValue = deferredReader(slot)->load(std::memory_order_relaxed);
269         assert(!slotValueIsThis(slotValue));
270       }
271     }
272 #endif
273   }
274
275   void lock() {
276     WaitForever ctx;
277     (void)lockExclusiveImpl(kHasSolo, ctx);
278   }
279
280   bool try_lock() {
281     WaitNever ctx;
282     return lockExclusiveImpl(kHasSolo, ctx);
283   }
284
285   template <class Rep, class Period>
286   bool try_lock_for(const std::chrono::duration<Rep, Period>& duration) {
287     WaitForDuration<Rep, Period> ctx(duration);
288     return lockExclusiveImpl(kHasSolo, ctx);
289   }
290
291   template <class Clock, class Duration>
292   bool try_lock_until(
293       const std::chrono::time_point<Clock, Duration>& absDeadline) {
294     WaitUntilDeadline<Clock, Duration> ctx{absDeadline};
295     return lockExclusiveImpl(kHasSolo, ctx);
296   }
297
298   void unlock() {
299     // It is possible that we have a left-over kWaitingNotS if the last
300     // unlock_shared() that let our matching lock() complete finished
301     // releasing before lock()'s futexWait went to sleep.  Clean it up now
302     auto state = (state_ &= ~(kWaitingNotS | kPrevDefer | kHasE));
303     assert((state & ~kWaitingAny) == 0);
304     wakeRegisteredWaiters(state, kWaitingE | kWaitingU | kWaitingS);
305   }
306
307   // Managing the token yourself makes unlock_shared a bit faster
308
309   void lock_shared() {
310     WaitForever ctx;
311     (void)lockSharedImpl(nullptr, ctx);
312   }
313
314   void lock_shared(Token& token) {
315     WaitForever ctx;
316     (void)lockSharedImpl(&token, ctx);
317   }
318
319   bool try_lock_shared() {
320     WaitNever ctx;
321     return lockSharedImpl(nullptr, ctx);
322   }
323
324   bool try_lock_shared(Token& token) {
325     WaitNever ctx;
326     return lockSharedImpl(&token, ctx);
327   }
328
329   template <class Rep, class Period>
330   bool try_lock_shared_for(const std::chrono::duration<Rep, Period>& duration) {
331     WaitForDuration<Rep, Period> ctx(duration);
332     return lockSharedImpl(nullptr, ctx);
333   }
334
335   template <class Rep, class Period>
336   bool try_lock_shared_for(const std::chrono::duration<Rep, Period>& duration,
337                            Token& token) {
338     WaitForDuration<Rep, Period> ctx(duration);
339     return lockSharedImpl(&token, ctx);
340   }
341
342   template <class Clock, class Duration>
343   bool try_lock_shared_until(
344       const std::chrono::time_point<Clock, Duration>& absDeadline) {
345     WaitUntilDeadline<Clock, Duration> ctx{absDeadline};
346     return lockSharedImpl(nullptr, ctx);
347   }
348
349   template <class Clock, class Duration>
350   bool try_lock_shared_until(
351       const std::chrono::time_point<Clock, Duration>& absDeadline,
352       Token& token) {
353     WaitUntilDeadline<Clock, Duration> ctx{absDeadline};
354     return lockSharedImpl(&token, ctx);
355   }
356
357   void unlock_shared() {
358     auto state = state_.load(std::memory_order_acquire);
359
360     // kPrevDefer can only be set if HasE or BegunE is set
361     assert((state & (kPrevDefer | kHasE | kBegunE)) != kPrevDefer);
362
363     // lock() strips kMayDefer immediately, but then copies it to
364     // kPrevDefer so we can tell if the pre-lock() lock_shared() might
365     // have deferred
366     if ((state & (kMayDefer | kPrevDefer)) == 0 ||
367         !tryUnlockTokenlessSharedDeferred()) {
368       // Matching lock_shared() couldn't have deferred, or the deferred
369       // lock has already been inlined by applyDeferredReaders()
370       unlockSharedInline();
371     }
372   }
373
374   void unlock_shared(Token& token) {
375     assert(token.type_ == Token::Type::INLINE_SHARED ||
376            token.type_ == Token::Type::DEFERRED_SHARED);
377
378     if (token.type_ != Token::Type::DEFERRED_SHARED ||
379         !tryUnlockSharedDeferred(token.slot_)) {
380       unlockSharedInline();
381     }
382 #ifndef NDEBUG
383     token.type_ = Token::Type::INVALID;
384 #endif
385   }
386
387   void unlock_and_lock_shared() {
388     // We can't use state_ -=, because we need to clear 2 bits (1 of which
389     // has an uncertain initial state) and set 1 other.  We might as well
390     // clear the relevant wake bits at the same time.  Note that since S
391     // doesn't block the beginning of a transition to E (writer priority
392     // can cut off new S, reader priority grabs BegunE and blocks deferred
393     // S) we need to wake E as well.
394     auto state = state_.load(std::memory_order_acquire);
395     do {
396       assert((state & ~(kWaitingAny | kPrevDefer)) == kHasE);
397     } while (!state_.compare_exchange_strong(
398         state, (state & ~(kWaitingAny | kPrevDefer | kHasE)) + kIncrHasS));
399     if ((state & (kWaitingE | kWaitingU | kWaitingS)) != 0) {
400       futexWakeAll(kWaitingE | kWaitingU | kWaitingS);
401     }
402   }
403
404   void unlock_and_lock_shared(Token& token) {
405     unlock_and_lock_shared();
406     token.type_ = Token::Type::INLINE_SHARED;
407   }
408
409   void lock_upgrade() {
410     WaitForever ctx;
411     (void)lockUpgradeImpl(ctx);
412   }
413
414   bool try_lock_upgrade() {
415     WaitNever ctx;
416     return lockUpgradeImpl(ctx);
417   }
418
419   template <class Rep, class Period>
420   bool try_lock_upgrade_for(
421       const std::chrono::duration<Rep, Period>& duration) {
422     WaitForDuration<Rep, Period> ctx(duration);
423     return lockUpgradeImpl(ctx);
424   }
425
426   template <class Clock, class Duration>
427   bool try_lock_upgrade_until(
428       const std::chrono::time_point<Clock, Duration>& absDeadline) {
429     WaitUntilDeadline<Clock, Duration> ctx{absDeadline};
430     return lockUpgradeImpl(ctx);
431   }
432
433   void unlock_upgrade() {
434     auto state = (state_ -= kHasU);
435     assert((state & (kWaitingNotS | kHasSolo)) == 0);
436     wakeRegisteredWaiters(state, kWaitingE | kWaitingU);
437   }
438
439   void unlock_upgrade_and_lock() {
440     // no waiting necessary, so waitMask is empty
441     WaitForever ctx;
442     (void)lockExclusiveImpl(0, ctx);
443   }
444
445   void unlock_upgrade_and_lock_shared() {
446     auto state = (state_ -= kHasU - kIncrHasS);
447     assert((state & (kWaitingNotS | kHasSolo)) == 0);
448     wakeRegisteredWaiters(state, kWaitingE | kWaitingU);
449   }
450
451   void unlock_upgrade_and_lock_shared(Token& token) {
452     unlock_upgrade_and_lock_shared();
453     token.type_ = Token::Type::INLINE_SHARED;
454   }
455
456   void unlock_and_lock_upgrade() {
457     // We can't use state_ -=, because we need to clear 2 bits (1 of
458     // which has an uncertain initial state) and set 1 other.  We might
459     // as well clear the relevant wake bits at the same time.
460     auto state = state_.load(std::memory_order_acquire);
461     while (true) {
462       assert((state & ~(kWaitingAny | kPrevDefer)) == kHasE);
463       auto after =
464           (state & ~(kWaitingNotS | kWaitingS | kPrevDefer | kHasE)) + kHasU;
465       if (state_.compare_exchange_strong(state, after)) {
466         if ((state & kWaitingS) != 0) {
467           futexWakeAll(kWaitingS);
468         }
469         return;
470       }
471     }
472   }
473
474  private:
475   typedef typename folly::detail::Futex<Atom> Futex;
476
477   // Internally we use four kinds of wait contexts.  These are structs
478   // that provide a doWait method that returns true if a futex wake
479   // was issued that intersects with the waitMask, false if there was a
480   // timeout and no more waiting should be performed.  Spinning occurs
481   // before the wait context is invoked.
482
483   struct WaitForever {
484     bool canBlock() { return true; }
485     bool canTimeOut() { return false; }
486     bool shouldTimeOut() { return false; }
487
488     bool doWait(Futex& futex, uint32_t expected, uint32_t waitMask) {
489       futex.futexWait(expected, waitMask);
490       return true;
491     }
492   };
493
494   struct WaitNever {
495     bool canBlock() { return false; }
496     bool canTimeOut() { return true; }
497     bool shouldTimeOut() { return true; }
498
499     bool doWait(Futex& /* futex */,
500                 uint32_t /* expected */,
501                 uint32_t /* waitMask */) {
502       return false;
503     }
504   };
505
506   template <class Rep, class Period>
507   struct WaitForDuration {
508     std::chrono::duration<Rep, Period> duration_;
509     bool deadlineComputed_;
510     std::chrono::steady_clock::time_point deadline_;
511
512     explicit WaitForDuration(const std::chrono::duration<Rep, Period>& duration)
513         : duration_(duration), deadlineComputed_(false) {}
514
515     std::chrono::steady_clock::time_point deadline() {
516       if (!deadlineComputed_) {
517         deadline_ = std::chrono::steady_clock::now() + duration_;
518         deadlineComputed_ = true;
519       }
520       return deadline_;
521     }
522
523     bool canBlock() { return duration_.count() > 0; }
524     bool canTimeOut() { return true; }
525
526     bool shouldTimeOut() {
527       return std::chrono::steady_clock::now() > deadline();
528     }
529
530     bool doWait(Futex& futex, uint32_t expected, uint32_t waitMask) {
531       auto result = futex.futexWaitUntil(expected, deadline(), waitMask);
532       return result != folly::detail::FutexResult::TIMEDOUT;
533     }
534   };
535
536   template <class Clock, class Duration>
537   struct WaitUntilDeadline {
538     std::chrono::time_point<Clock, Duration> absDeadline_;
539
540     bool canBlock() { return true; }
541     bool canTimeOut() { return true; }
542     bool shouldTimeOut() { return Clock::now() > absDeadline_; }
543
544     bool doWait(Futex& futex, uint32_t expected, uint32_t waitMask) {
545       auto result = futex.futexWaitUntil(expected, absDeadline_, waitMask);
546       return result != folly::detail::FutexResult::TIMEDOUT;
547     }
548   };
549
550   // 32 bits of state
551   Futex state_;
552
553   // S count needs to be on the end, because we explicitly allow it to
554   // underflow.  This can occur while we are in the middle of applying
555   // deferred locks (we remove them from deferredReaders[] before
556   // inlining them), or during token-less unlock_shared() if a racing
557   // lock_shared();unlock_shared() moves the deferredReaders slot while
558   // the first unlock_shared() is scanning.  The former case is cleaned
559   // up before we finish applying the locks.  The latter case can persist
560   // until destruction, when it is cleaned up.
561   static constexpr uint32_t kIncrHasS = 1 << 10;
562   static constexpr uint32_t kHasS = ~(kIncrHasS - 1);
563
564   // If false, then there are definitely no deferred read locks for this
565   // instance.  Cleared after initialization and when exclusively locked.
566   static constexpr uint32_t kMayDefer = 1 << 9;
567
568   // lock() cleared kMayDefer as soon as it starts draining readers (so
569   // that it doesn't have to do a second CAS once drain completes), but
570   // unlock_shared() still needs to know whether to scan deferredReaders[]
571   // or not.  We copy kMayDefer to kPrevDefer when setting kHasE or
572   // kBegunE, and clear it when clearing those bits.
573   static constexpr uint32_t kPrevDefer = 1 << 8;
574
575   // Exclusive-locked blocks all read locks and write locks.  This bit
576   // may be set before all readers have finished, but in that case the
577   // thread that sets it won't return to the caller until all read locks
578   // have been released.
579   static constexpr uint32_t kHasE = 1 << 7;
580
581   // Exclusive-draining means that lock() is waiting for existing readers
582   // to leave, but that new readers may still acquire shared access.
583   // This is only used in reader priority mode.  New readers during
584   // drain must be inline.  The difference between this and kHasU is that
585   // kBegunE prevents kMayDefer from being set.
586   static constexpr uint32_t kBegunE = 1 << 6;
587
588   // At most one thread may have either exclusive or upgrade lock
589   // ownership.  Unlike exclusive mode, ownership of the lock in upgrade
590   // mode doesn't preclude other threads holding the lock in shared mode.
591   // boost's concept for this doesn't explicitly say whether new shared
592   // locks can be acquired one lock_upgrade has succeeded, but doesn't
593   // list that as disallowed.  RWSpinLock disallows new read locks after
594   // lock_upgrade has been acquired, but the boost implementation doesn't.
595   // We choose the latter.
596   static constexpr uint32_t kHasU = 1 << 5;
597
598   // There are three states that we consider to be "solo", in that they
599   // cannot coexist with other solo states.  These are kHasE, kBegunE,
600   // and kHasU.  Note that S doesn't conflict with any of these, because
601   // setting the kHasE is only one of the two steps needed to actually
602   // acquire the lock in exclusive mode (the other is draining the existing
603   // S holders).
604   static constexpr uint32_t kHasSolo = kHasE | kBegunE | kHasU;
605
606   // Once a thread sets kHasE it needs to wait for the current readers
607   // to exit the lock.  We give this a separate wait identity from the
608   // waiting to set kHasE so that we can perform partial wakeups (wake
609   // one instead of wake all).
610   static constexpr uint32_t kWaitingNotS = 1 << 4;
611
612   // When waking writers we can either wake them all, in which case we
613   // can clear kWaitingE, or we can call futexWake(1).  futexWake tells
614   // us if anybody woke up, but even if we detect that nobody woke up we
615   // can't clear the bit after the fact without issuing another wakeup.
616   // To avoid thundering herds when there are lots of pending lock()
617   // without needing to call futexWake twice when there is only one
618   // waiter, kWaitingE actually encodes if we have observed multiple
619   // concurrent waiters.  Tricky: ABA issues on futexWait mean that when
620   // we see kWaitingESingle we can't assume that there is only one.
621   static constexpr uint32_t kWaitingESingle = 1 << 2;
622   static constexpr uint32_t kWaitingEMultiple = 1 << 3;
623   static constexpr uint32_t kWaitingE = kWaitingESingle | kWaitingEMultiple;
624
625   // kWaitingU is essentially a 1 bit saturating counter.  It always
626   // requires a wakeAll.
627   static constexpr uint32_t kWaitingU = 1 << 1;
628
629   // All blocked lock_shared() should be awoken, so it is correct (not
630   // suboptimal) to wakeAll if there are any shared readers.
631   static constexpr uint32_t kWaitingS = 1 << 0;
632
633   // kWaitingAny is a mask of all of the bits that record the state of
634   // threads, rather than the state of the lock.  It is convenient to be
635   // able to mask them off during asserts.
636   static constexpr uint32_t kWaitingAny =
637       kWaitingNotS | kWaitingE | kWaitingU | kWaitingS;
638
639   // The reader count at which a reader will attempt to use the lock
640   // in deferred mode.  If this value is 2, then the second concurrent
641   // reader will set kMayDefer and use deferredReaders[].  kMayDefer is
642   // cleared during exclusive access, so this threshold must be reached
643   // each time a lock is held in exclusive mode.
644   static constexpr uint32_t kNumSharedToStartDeferring = 2;
645
646   // The typical number of spins that a thread will wait for a state
647   // transition.  There is no bound on the number of threads that can wait
648   // for a writer, so we are pretty conservative here to limit the chance
649   // that we are starving the writer of CPU.  Each spin is 6 or 7 nanos,
650   // almost all of which is in the pause instruction.
651   static constexpr uint32_t kMaxSpinCount = !BlockImmediately ? 1000 : 2;
652
653   // The maximum number of soft yields before falling back to futex.
654   // If the preemption heuristic is activated we will fall back before
655   // this.  A soft yield takes ~900 nanos (two sched_yield plus a call
656   // to getrusage, with checks of the goal at each step).  Soft yields
657   // aren't compatible with deterministic execution under test (unlike
658   // futexWaitUntil, which has a capricious but deterministic back end).
659   static constexpr uint32_t kMaxSoftYieldCount = !BlockImmediately ? 1000 : 0;
660
661   // If AccessSpreader assigns indexes from 0..k*n-1 on a system where some
662   // level of the memory hierarchy is symmetrically divided into k pieces
663   // (NUMA nodes, last-level caches, L1 caches, ...), then slot indexes
664   // that are the same after integer division by k share that resource.
665   // Our strategy for deferred readers is to probe up to numSlots/4 slots,
666   // using the full granularity of AccessSpreader for the start slot
667   // and then search outward.  We can use AccessSpreader::current(n)
668   // without managing our own spreader if kMaxDeferredReaders <=
669   // AccessSpreader::kMaxCpus, which is currently 128.
670   //
671   // Our 2-socket E5-2660 machines have 8 L1 caches on each chip,
672   // with 64 byte cache lines.  That means we need 64*16 bytes of
673   // deferredReaders[] to give each L1 its own playground.  On x86_64
674   // each DeferredReaderSlot is 8 bytes, so we need kMaxDeferredReaders
675   // * kDeferredSeparationFactor >= 64 * 16 / 8 == 128.  If
676   // kDeferredSearchDistance * kDeferredSeparationFactor <=
677   // 64 / 8 then we will search only within a single cache line, which
678   // guarantees we won't have inter-L1 contention.  We give ourselves
679   // a factor of 2 on the core count, which should hold us for a couple
680   // processor generations.  deferredReaders[] is 2048 bytes currently.
681  public:
682   static constexpr uint32_t kMaxDeferredReaders = 64;
683   static constexpr uint32_t kDeferredSearchDistance = 2;
684   static constexpr uint32_t kDeferredSeparationFactor = 4;
685
686  private:
687
688   static_assert(!(kMaxDeferredReaders & (kMaxDeferredReaders - 1)),
689                 "kMaxDeferredReaders must be a power of 2");
690   static_assert(!(kDeferredSearchDistance & (kDeferredSearchDistance - 1)),
691                 "kDeferredSearchDistance must be a power of 2");
692
693   // The number of deferred locks that can be simultaneously acquired
694   // by a thread via the token-less methods without performing any heap
695   // allocations.  Each of these costs 3 pointers (24 bytes, probably)
696   // per thread.  There's not much point in making this larger than
697   // kDeferredSearchDistance.
698   static constexpr uint32_t kTokenStackTLSCapacity = 2;
699
700   // We need to make sure that if there is a lock_shared()
701   // and lock_shared(token) followed by unlock_shared() and
702   // unlock_shared(token), the token-less unlock doesn't null
703   // out deferredReaders[token.slot_].  If we allowed that, then
704   // unlock_shared(token) wouldn't be able to assume that its lock
705   // had been inlined by applyDeferredReaders when it finds that
706   // deferredReaders[token.slot_] no longer points to this.  We accomplish
707   // this by stealing bit 0 from the pointer to record that the slot's
708   // element has no token, hence our use of uintptr_t in deferredReaders[].
709   static constexpr uintptr_t kTokenless = 0x1;
710
711   // This is the starting location for Token-less unlock_shared().
712   static FOLLY_TLS uint32_t tls_lastTokenlessSlot;
713
714   // Only indexes divisible by kDeferredSeparationFactor are used.
715   // If any of those elements points to a SharedMutexImpl, then it
716   // should be considered that there is a shared lock on that instance.
717   // See kTokenless.
718  public:
719   typedef Atom<uintptr_t> DeferredReaderSlot;
720
721  private:
722   FOLLY_ALIGN_TO_AVOID_FALSE_SHARING static DeferredReaderSlot deferredReaders
723       [kMaxDeferredReaders *
724        kDeferredSeparationFactor];
725
726   // Performs an exclusive lock, waiting for state_ & waitMask to be
727   // zero first
728   template <class WaitContext>
729   bool lockExclusiveImpl(uint32_t preconditionGoalMask, WaitContext& ctx) {
730     uint32_t state = state_.load(std::memory_order_acquire);
731     if (LIKELY(
732             (state & (preconditionGoalMask | kMayDefer | kHasS)) == 0 &&
733             state_.compare_exchange_strong(state, (state | kHasE) & ~kHasU))) {
734       return true;
735     } else {
736       return lockExclusiveImpl(state, preconditionGoalMask, ctx);
737     }
738   }
739
740   template <class WaitContext>
741   bool lockExclusiveImpl(uint32_t& state,
742                          uint32_t preconditionGoalMask,
743                          WaitContext& ctx) {
744     while (true) {
745       if (UNLIKELY((state & preconditionGoalMask) != 0) &&
746           !waitForZeroBits(state, preconditionGoalMask, kWaitingE, ctx) &&
747           ctx.canTimeOut()) {
748         return false;
749       }
750
751       uint32_t after = (state & kMayDefer) == 0 ? 0 : kPrevDefer;
752       if (!ReaderPriority || (state & (kMayDefer | kHasS)) == 0) {
753         // Block readers immediately, either because we are in write
754         // priority mode or because we can acquire the lock in one
755         // step.  Note that if state has kHasU, then we are doing an
756         // unlock_upgrade_and_lock() and we should clear it (reader
757         // priority branch also does this).
758         after |= (state | kHasE) & ~(kHasU | kMayDefer);
759       } else {
760         after |= (state | kBegunE) & ~(kHasU | kMayDefer);
761       }
762       if (state_.compare_exchange_strong(state, after)) {
763         auto before = state;
764         state = after;
765
766         // If we set kHasE (writer priority) then no new readers can
767         // arrive.  If we set kBegunE then they can still enter, but
768         // they must be inline.  Either way we need to either spin on
769         // deferredReaders[] slots, or inline them so that we can wait on
770         // kHasS to zero itself.  deferredReaders[] is pointers, which on
771         // x86_64 are bigger than futex() can handle, so we inline the
772         // deferred locks instead of trying to futexWait on each slot.
773         // Readers are responsible for rechecking state_ after recording
774         // a deferred read to avoid atomicity problems between the state_
775         // CAS and applyDeferredReader's reads of deferredReaders[].
776         if (UNLIKELY((before & kMayDefer) != 0)) {
777           applyDeferredReaders(state, ctx);
778         }
779         while (true) {
780           assert((state & (kHasE | kBegunE)) != 0 && (state & kHasU) == 0);
781           if (UNLIKELY((state & kHasS) != 0) &&
782               !waitForZeroBits(state, kHasS, kWaitingNotS, ctx) &&
783               ctx.canTimeOut()) {
784             // Ugh.  We blocked new readers and other writers for a while,
785             // but were unable to complete.  Move on.  On the plus side
786             // we can clear kWaitingNotS because nobody else can piggyback
787             // on it.
788             state = (state_ &= ~(kPrevDefer | kHasE | kBegunE | kWaitingNotS));
789             wakeRegisteredWaiters(state, kWaitingE | kWaitingU | kWaitingS);
790             return false;
791           }
792
793           if (ReaderPriority && (state & kHasE) == 0) {
794             assert((state & kBegunE) != 0);
795             if (!state_.compare_exchange_strong(state,
796                                                 (state & ~kBegunE) | kHasE)) {
797               continue;
798             }
799           }
800
801           return true;
802         }
803       }
804     }
805   }
806
807   template <class WaitContext>
808   bool waitForZeroBits(uint32_t& state,
809                        uint32_t goal,
810                        uint32_t waitMask,
811                        WaitContext& ctx) {
812     uint32_t spinCount = 0;
813     while (true) {
814       state = state_.load(std::memory_order_acquire);
815       if ((state & goal) == 0) {
816         return true;
817       }
818       asm_volatile_pause();
819       ++spinCount;
820       if (UNLIKELY(spinCount >= kMaxSpinCount)) {
821         return ctx.canBlock() &&
822                yieldWaitForZeroBits(state, goal, waitMask, ctx);
823       }
824     }
825   }
826
827   template <class WaitContext>
828   bool yieldWaitForZeroBits(uint32_t& state,
829                             uint32_t goal,
830                             uint32_t waitMask,
831                             WaitContext& ctx) {
832 #ifdef RUSAGE_THREAD
833     struct rusage usage;
834     long before = -1;
835 #endif
836     for (uint32_t yieldCount = 0; yieldCount < kMaxSoftYieldCount;
837          ++yieldCount) {
838       for (int softState = 0; softState < 3; ++softState) {
839         if (softState < 2) {
840           std::this_thread::yield();
841         } else {
842 #ifdef RUSAGE_THREAD
843           getrusage(RUSAGE_THREAD, &usage);
844 #endif
845         }
846         if (((state = state_.load(std::memory_order_acquire)) & goal) == 0) {
847           return true;
848         }
849         if (ctx.shouldTimeOut()) {
850           return false;
851         }
852       }
853 #ifdef RUSAGE_THREAD
854       if (before >= 0 && usage.ru_nivcsw >= before + 2) {
855         // One involuntary csw might just be occasional background work,
856         // but if we get two in a row then we guess that there is someone
857         // else who can profitably use this CPU.  Fall back to futex
858         break;
859       }
860       before = usage.ru_nivcsw;
861 #endif
862     }
863     return futexWaitForZeroBits(state, goal, waitMask, ctx);
864   }
865
866   template <class WaitContext>
867   bool futexWaitForZeroBits(uint32_t& state,
868                             uint32_t goal,
869                             uint32_t waitMask,
870                             WaitContext& ctx) {
871     assert(waitMask == kWaitingNotS || waitMask == kWaitingE ||
872            waitMask == kWaitingU || waitMask == kWaitingS);
873
874     while (true) {
875       state = state_.load(std::memory_order_acquire);
876       if ((state & goal) == 0) {
877         return true;
878       }
879
880       auto after = state;
881       if (waitMask == kWaitingE) {
882         if ((state & kWaitingESingle) != 0) {
883           after |= kWaitingEMultiple;
884         } else {
885           after |= kWaitingESingle;
886         }
887       } else {
888         after |= waitMask;
889       }
890
891       // CAS is better than atomic |= here, because it lets us avoid
892       // setting the wait flag when the goal is concurrently achieved
893       if (after != state && !state_.compare_exchange_strong(state, after)) {
894         continue;
895       }
896
897       if (!ctx.doWait(state_, after, waitMask)) {
898         // timed out
899         return false;
900       }
901     }
902   }
903
904   // Wakes up waiters registered in state_ as appropriate, clearing the
905   // awaiting bits for anybody that was awoken.  Tries to perform direct
906   // single wakeup of an exclusive waiter if appropriate
907   void wakeRegisteredWaiters(uint32_t& state, uint32_t wakeMask) {
908     if (UNLIKELY((state & wakeMask) != 0)) {
909       wakeRegisteredWaitersImpl(state, wakeMask);
910     }
911   }
912
913   void wakeRegisteredWaitersImpl(uint32_t& state, uint32_t wakeMask) {
914     // If there are multiple lock() pending only one of them will actually
915     // get to wake up, so issuing futexWakeAll will make a thundering herd.
916     // There's nothing stopping us from issuing futexWake(1) instead,
917     // so long as the wait bits are still an accurate reflection of
918     // the waiters.  If we notice (via futexWake's return value) that
919     // nobody woke up then we can try again with the normal wake-all path.
920     // Note that we can't just clear the bits at that point; we need to
921     // clear the bits and then issue another wakeup.
922     //
923     // It is possible that we wake an E waiter but an outside S grabs the
924     // lock instead, at which point we should wake pending U and S waiters.
925     // Rather than tracking state to make the failing E regenerate the
926     // wakeup, we just disable the optimization in the case that there
927     // are waiting U or S that we are eligible to wake.
928     if ((wakeMask & kWaitingE) == kWaitingE &&
929         (state & wakeMask) == kWaitingE &&
930         state_.futexWake(1, kWaitingE) > 0) {
931       // somebody woke up, so leave state_ as is and clear it later
932       return;
933     }
934
935     if ((state & wakeMask) != 0) {
936       auto prev = state_.fetch_and(~wakeMask);
937       if ((prev & wakeMask) != 0) {
938         futexWakeAll(wakeMask);
939       }
940       state = prev & ~wakeMask;
941     }
942   }
943
944   void futexWakeAll(uint32_t wakeMask) {
945     state_.futexWake(std::numeric_limits<int>::max(), wakeMask);
946   }
947
948   DeferredReaderSlot* deferredReader(uint32_t slot) {
949     return &deferredReaders[slot * kDeferredSeparationFactor];
950   }
951
952   uintptr_t tokenfulSlotValue() { return reinterpret_cast<uintptr_t>(this); }
953
954   uintptr_t tokenlessSlotValue() { return tokenfulSlotValue() | kTokenless; }
955
956   bool slotValueIsThis(uintptr_t slotValue) {
957     return (slotValue & ~kTokenless) == tokenfulSlotValue();
958   }
959
960   // Clears any deferredReaders[] that point to this, adjusting the inline
961   // shared lock count to compensate.  Does some spinning and yielding
962   // to avoid the work.  Always finishes the application, even if ctx
963   // times out.
964   template <class WaitContext>
965   void applyDeferredReaders(uint32_t& state, WaitContext& ctx) {
966     uint32_t slot = 0;
967
968     uint32_t spinCount = 0;
969     while (true) {
970       while (!slotValueIsThis(
971                  deferredReader(slot)->load(std::memory_order_acquire))) {
972         if (++slot == kMaxDeferredReaders) {
973           return;
974         }
975       }
976       asm_pause();
977       if (UNLIKELY(++spinCount >= kMaxSpinCount)) {
978         applyDeferredReaders(state, ctx, slot);
979         return;
980       }
981     }
982   }
983
984   template <class WaitContext>
985   void applyDeferredReaders(uint32_t& state, WaitContext& ctx, uint32_t slot) {
986
987 #ifdef RUSAGE_THREAD
988     struct rusage usage;
989     long before = -1;
990 #endif
991     for (uint32_t yieldCount = 0; yieldCount < kMaxSoftYieldCount;
992          ++yieldCount) {
993       for (int softState = 0; softState < 3; ++softState) {
994         if (softState < 2) {
995           std::this_thread::yield();
996         } else {
997 #ifdef RUSAGE_THREAD
998           getrusage(RUSAGE_THREAD, &usage);
999 #endif
1000         }
1001         while (!slotValueIsThis(
1002                    deferredReader(slot)->load(std::memory_order_acquire))) {
1003           if (++slot == kMaxDeferredReaders) {
1004             return;
1005           }
1006         }
1007         if (ctx.shouldTimeOut()) {
1008           // finish applying immediately on timeout
1009           break;
1010         }
1011       }
1012 #ifdef RUSAGE_THREAD
1013       if (before >= 0 && usage.ru_nivcsw >= before + 2) {
1014         // heuristic says run queue is not empty
1015         break;
1016       }
1017       before = usage.ru_nivcsw;
1018 #endif
1019     }
1020
1021     uint32_t movedSlotCount = 0;
1022     for (; slot < kMaxDeferredReaders; ++slot) {
1023       auto slotPtr = deferredReader(slot);
1024       auto slotValue = slotPtr->load(std::memory_order_acquire);
1025       if (slotValueIsThis(slotValue) &&
1026           slotPtr->compare_exchange_strong(slotValue, 0)) {
1027         ++movedSlotCount;
1028       }
1029     }
1030
1031     if (movedSlotCount > 0) {
1032       state = (state_ += movedSlotCount * kIncrHasS);
1033     }
1034     assert((state & (kHasE | kBegunE)) != 0);
1035
1036     // if state + kIncrHasS overflows (off the end of state) then either
1037     // we have 2^(32-9) readers (almost certainly an application bug)
1038     // or we had an underflow (also a bug)
1039     assert(state < state + kIncrHasS);
1040   }
1041
1042   // It is straightfoward to make a token-less lock_shared() and
1043   // unlock_shared() either by making the token-less version always use
1044   // INLINE_SHARED mode or by removing the token version.  Supporting
1045   // deferred operation for both types is trickier than it appears, because
1046   // the purpose of the token it so that unlock_shared doesn't have to
1047   // look in other slots for its deferred lock.  Token-less unlock_shared
1048   // might place a deferred lock in one place and then release a different
1049   // slot that was originally used by the token-ful version.  If this was
1050   // important we could solve the problem by differentiating the deferred
1051   // locks so that cross-variety release wouldn't occur.  The best way
1052   // is probably to steal a bit from the pointer, making deferredLocks[]
1053   // an array of Atom<uintptr_t>.
1054
1055   template <class WaitContext>
1056   bool lockSharedImpl(Token* token, WaitContext& ctx) {
1057     uint32_t state = state_.load(std::memory_order_relaxed);
1058     if ((state & (kHasS | kMayDefer | kHasE)) == 0 &&
1059         state_.compare_exchange_strong(state, state + kIncrHasS)) {
1060       if (token != nullptr) {
1061         token->type_ = Token::Type::INLINE_SHARED;
1062       }
1063       return true;
1064     }
1065     return lockSharedImpl(state, token, ctx);
1066   }
1067
1068   template <class WaitContext>
1069   bool lockSharedImpl(uint32_t& state, Token* token, WaitContext& ctx) {
1070     while (true) {
1071       if (UNLIKELY((state & kHasE) != 0) &&
1072           !waitForZeroBits(state, kHasE, kWaitingS, ctx) && ctx.canTimeOut()) {
1073         return false;
1074       }
1075
1076       uint32_t slot;
1077       uintptr_t slotValue = 1; // any non-zero value will do
1078
1079       bool canAlreadyDefer = (state & kMayDefer) != 0;
1080       bool aboveDeferThreshold =
1081           (state & kHasS) >= (kNumSharedToStartDeferring - 1) * kIncrHasS;
1082       bool drainInProgress = ReaderPriority && (state & kBegunE) != 0;
1083       if (canAlreadyDefer || (aboveDeferThreshold && !drainInProgress)) {
1084         // starting point for our empty-slot search, can change after
1085         // calling waitForZeroBits
1086         uint32_t bestSlot =
1087             (uint32_t)folly::detail::AccessSpreader<Atom>::current(
1088                 kMaxDeferredReaders);
1089
1090         // deferred readers are already enabled, or it is time to
1091         // enable them if we can find a slot
1092         for (uint32_t i = 0; i < kDeferredSearchDistance; ++i) {
1093           slot = bestSlot ^ i;
1094           assert(slot < kMaxDeferredReaders);
1095           slotValue = deferredReader(slot)->load(std::memory_order_relaxed);
1096           if (slotValue == 0) {
1097             // found empty slot
1098             break;
1099           }
1100         }
1101       }
1102
1103       if (slotValue != 0) {
1104         // not yet deferred, or no empty slots
1105         if (state_.compare_exchange_strong(state, state + kIncrHasS)) {
1106           // successfully recorded the read lock inline
1107           if (token != nullptr) {
1108             token->type_ = Token::Type::INLINE_SHARED;
1109           }
1110           return true;
1111         }
1112         // state is updated, try again
1113         continue;
1114       }
1115
1116       // record that deferred readers might be in use if necessary
1117       if ((state & kMayDefer) == 0) {
1118         if (!state_.compare_exchange_strong(state, state | kMayDefer)) {
1119           // keep going if CAS failed because somebody else set the bit
1120           // for us
1121           if ((state & (kHasE | kMayDefer)) != kMayDefer) {
1122             continue;
1123           }
1124         }
1125         // state = state | kMayDefer;
1126       }
1127
1128       // try to use the slot
1129       bool gotSlot = deferredReader(slot)->compare_exchange_strong(
1130           slotValue,
1131           token == nullptr ? tokenlessSlotValue() : tokenfulSlotValue());
1132
1133       // If we got the slot, we need to verify that an exclusive lock
1134       // didn't happen since we last checked.  If we didn't get the slot we
1135       // need to recheck state_ anyway to make sure we don't waste too much
1136       // work.  It is also possible that since we checked state_ someone
1137       // has acquired and released the write lock, clearing kMayDefer.
1138       // Both cases are covered by looking for the readers-possible bit,
1139       // because it is off when the exclusive lock bit is set.
1140       state = state_.load(std::memory_order_acquire);
1141
1142       if (!gotSlot) {
1143         continue;
1144       }
1145
1146       if (token == nullptr) {
1147         tls_lastTokenlessSlot = slot;
1148       }
1149
1150       if ((state & kMayDefer) != 0) {
1151         assert((state & kHasE) == 0);
1152         // success
1153         if (token != nullptr) {
1154           token->type_ = Token::Type::DEFERRED_SHARED;
1155           token->slot_ = (uint16_t)slot;
1156         }
1157         return true;
1158       }
1159
1160       // release the slot before retrying
1161       if (token == nullptr) {
1162         // We can't rely on slot.  Token-less slot values can be freed by
1163         // any unlock_shared(), so we need to do the full deferredReader
1164         // search during unlock.  Unlike unlock_shared(), we can't trust
1165         // kPrevDefer here.  This deferred lock isn't visible to lock()
1166         // (that's the whole reason we're undoing it) so there might have
1167         // subsequently been an unlock() and lock() with no intervening
1168         // transition to deferred mode.
1169         if (!tryUnlockTokenlessSharedDeferred()) {
1170           unlockSharedInline();
1171         }
1172       } else {
1173         if (!tryUnlockSharedDeferred(slot)) {
1174           unlockSharedInline();
1175         }
1176       }
1177
1178       // We got here not because the lock was unavailable, but because
1179       // we lost a compare-and-swap.  Try-lock is typically allowed to
1180       // have spurious failures, but there is no lock efficiency gain
1181       // from exploiting that freedom here.
1182     }
1183   }
1184
1185   // Updates the state in/out argument as if the locks were made inline,
1186   // but does not update state_
1187   void cleanupTokenlessSharedDeferred(uint32_t& state) {
1188     for (uint32_t i = 0; i < kMaxDeferredReaders; ++i) {
1189       auto slotPtr = deferredReader(i);
1190       auto slotValue = slotPtr->load(std::memory_order_relaxed);
1191       if (slotValue == tokenlessSlotValue()) {
1192         slotPtr->store(0, std::memory_order_relaxed);
1193         state += kIncrHasS;
1194         if ((state & kHasS) == 0) {
1195           break;
1196         }
1197       }
1198     }
1199   }
1200
1201   bool tryUnlockTokenlessSharedDeferred() {
1202     auto bestSlot = tls_lastTokenlessSlot;
1203     for (uint32_t i = 0; i < kMaxDeferredReaders; ++i) {
1204       auto slotPtr = deferredReader(bestSlot ^ i);
1205       auto slotValue = slotPtr->load(std::memory_order_relaxed);
1206       if (slotValue == tokenlessSlotValue() &&
1207           slotPtr->compare_exchange_strong(slotValue, 0)) {
1208         tls_lastTokenlessSlot = bestSlot ^ i;
1209         return true;
1210       }
1211     }
1212     return false;
1213   }
1214
1215   bool tryUnlockSharedDeferred(uint32_t slot) {
1216     assert(slot < kMaxDeferredReaders);
1217     auto slotValue = tokenfulSlotValue();
1218     return deferredReader(slot)->compare_exchange_strong(slotValue, 0);
1219   }
1220
1221   uint32_t unlockSharedInline() {
1222     uint32_t state = (state_ -= kIncrHasS);
1223     assert((state & (kHasE | kBegunE | kMayDefer)) != 0 ||
1224            state < state + kIncrHasS);
1225     if ((state & kHasS) == 0) {
1226       // Only the second half of lock() can be blocked by a non-zero
1227       // reader count, so that's the only thing we need to wake
1228       wakeRegisteredWaiters(state, kWaitingNotS);
1229     }
1230     return state;
1231   }
1232
1233   template <class WaitContext>
1234   bool lockUpgradeImpl(WaitContext& ctx) {
1235     uint32_t state;
1236     do {
1237       if (!waitForZeroBits(state, kHasSolo, kWaitingU, ctx)) {
1238         return false;
1239       }
1240     } while (!state_.compare_exchange_strong(state, state | kHasU));
1241     return true;
1242   }
1243
1244  public:
1245   class ReadHolder {
1246    public:
1247     ReadHolder() : lock_(nullptr) {}
1248
1249     explicit ReadHolder(const SharedMutexImpl* lock) : ReadHolder(*lock) {}
1250
1251     explicit ReadHolder(const SharedMutexImpl& lock)
1252         : lock_(const_cast<SharedMutexImpl*>(&lock)) {
1253       lock_->lock_shared(token_);
1254     }
1255
1256     ReadHolder(ReadHolder&& rhs) noexcept : lock_(rhs.lock_),
1257                                             token_(rhs.token_) {
1258       rhs.lock_ = nullptr;
1259     }
1260
1261     // Downgrade from upgrade mode
1262     explicit ReadHolder(UpgradeHolder&& upgraded) : lock_(upgraded.lock_) {
1263       assert(upgraded.lock_ != nullptr);
1264       upgraded.lock_ = nullptr;
1265       lock_->unlock_upgrade_and_lock_shared(token_);
1266     }
1267
1268     // Downgrade from exclusive mode
1269     explicit ReadHolder(WriteHolder&& writer) : lock_(writer.lock_) {
1270       assert(writer.lock_ != nullptr);
1271       writer.lock_ = nullptr;
1272       lock_->unlock_and_lock_shared(token_);
1273     }
1274
1275     ReadHolder& operator=(ReadHolder&& rhs) noexcept {
1276       std::swap(lock_, rhs.lock_);
1277       std::swap(token_, rhs.token_);
1278       return *this;
1279     }
1280
1281     ReadHolder(const ReadHolder& rhs) = delete;
1282     ReadHolder& operator=(const ReadHolder& rhs) = delete;
1283
1284     ~ReadHolder() {
1285       if (lock_) {
1286         lock_->unlock_shared(token_);
1287       }
1288     }
1289
1290    private:
1291     friend class UpgradeHolder;
1292     friend class WriteHolder;
1293     SharedMutexImpl* lock_;
1294     SharedMutexToken token_;
1295   };
1296
1297   class UpgradeHolder {
1298    public:
1299     UpgradeHolder() : lock_(nullptr) {}
1300
1301     explicit UpgradeHolder(SharedMutexImpl* lock) : UpgradeHolder(*lock) {}
1302
1303     explicit UpgradeHolder(SharedMutexImpl& lock) : lock_(&lock) {
1304       lock_->lock_upgrade();
1305     }
1306
1307     // Downgrade from exclusive mode
1308     explicit UpgradeHolder(WriteHolder&& writer) : lock_(writer.lock_) {
1309       assert(writer.lock_ != nullptr);
1310       writer.lock_ = nullptr;
1311       lock_->unlock_and_lock_upgrade();
1312     }
1313
1314     UpgradeHolder(UpgradeHolder&& rhs) noexcept : lock_(rhs.lock_) {
1315       rhs.lock_ = nullptr;
1316     }
1317
1318     UpgradeHolder& operator=(UpgradeHolder&& rhs) noexcept {
1319       std::swap(lock_, rhs.lock_);
1320       return *this;
1321     }
1322
1323     UpgradeHolder(const UpgradeHolder& rhs) = delete;
1324     UpgradeHolder& operator=(const UpgradeHolder& rhs) = delete;
1325
1326     ~UpgradeHolder() {
1327       if (lock_) {
1328         lock_->unlock_upgrade();
1329       }
1330     }
1331
1332    private:
1333     friend class WriteHolder;
1334     friend class ReadHolder;
1335     SharedMutexImpl* lock_;
1336   };
1337
1338   class WriteHolder {
1339    public:
1340     WriteHolder() : lock_(nullptr) {}
1341
1342     explicit WriteHolder(SharedMutexImpl* lock) : WriteHolder(*lock) {}
1343
1344     explicit WriteHolder(SharedMutexImpl& lock) : lock_(&lock) {
1345       lock_->lock();
1346     }
1347
1348     // Promotion from upgrade mode
1349     explicit WriteHolder(UpgradeHolder&& upgrade) : lock_(upgrade.lock_) {
1350       assert(upgrade.lock_ != nullptr);
1351       upgrade.lock_ = nullptr;
1352       lock_->unlock_upgrade_and_lock();
1353     }
1354
1355     WriteHolder(WriteHolder&& rhs) noexcept : lock_(rhs.lock_) {
1356       rhs.lock_ = nullptr;
1357     }
1358
1359     WriteHolder& operator=(WriteHolder&& rhs) noexcept {
1360       std::swap(lock_, rhs.lock_);
1361       return *this;
1362     }
1363
1364     WriteHolder(const WriteHolder& rhs) = delete;
1365     WriteHolder& operator=(const WriteHolder& rhs) = delete;
1366
1367     ~WriteHolder() {
1368       if (lock_) {
1369         lock_->unlock();
1370       }
1371     }
1372
1373    private:
1374     friend class ReadHolder;
1375     friend class UpgradeHolder;
1376     SharedMutexImpl* lock_;
1377   };
1378
1379   // Adapters for Synchronized<>
1380   friend void acquireRead(SharedMutexImpl& lock) { lock.lock_shared(); }
1381   friend void acquireReadWrite(SharedMutexImpl& lock) { lock.lock(); }
1382   friend void releaseRead(SharedMutexImpl& lock) { lock.unlock_shared(); }
1383   friend void releaseReadWrite(SharedMutexImpl& lock) { lock.unlock(); }
1384   friend bool acquireRead(SharedMutexImpl& lock, unsigned int ms) {
1385     return lock.try_lock_shared_for(std::chrono::milliseconds(ms));
1386   }
1387   friend bool acquireReadWrite(SharedMutexImpl& lock, unsigned int ms) {
1388     return lock.try_lock_for(std::chrono::milliseconds(ms));
1389   }
1390 };
1391
1392 #define COMMON_CONCURRENCY_SHARED_MUTEX_DECLARE_STATIC_STORAGE(type) \
1393   template <>                                                        \
1394   type::DeferredReaderSlot                                           \
1395       type::deferredReaders[type::kMaxDeferredReaders *              \
1396                             type::kDeferredSeparationFactor] = {};   \
1397   template <>                                                        \
1398   FOLLY_TLS uint32_t type::tls_lastTokenlessSlot = 0;
1399
1400 typedef SharedMutexImpl<true> SharedMutexReadPriority;
1401 typedef SharedMutexImpl<false> SharedMutexWritePriority;
1402 typedef SharedMutexWritePriority SharedMutex;
1403
1404 } // namespace folly