Add MicroLock as an alternative to MicroSpinLock
authorDaniel Colascione <dancol@fb.com>
Fri, 4 Mar 2016 00:33:07 +0000 (16:33 -0800)
committerFacebook Github Bot 8 <facebook-github-bot-8-bot@fb.com>
Fri, 4 Mar 2016 00:35:20 +0000 (16:35 -0800)
Summary:MicroLock is a full-featured lock that fills the niche that
MicroSpinLock and PicoSpinLock currently inhabit.  Unlike these two
classes, MicroLock is a sleeping lock.  MicroLock requires two bits of
a single word and has no particular alignment requirements.

Reviewed By: ot

Differential Revision: D3004474

fb-gh-sync-id: a9bd28bd3f48e894d5bff407612ee4b228466209
shipit-source-id: a9bd28bd3f48e894d5bff407612ee4b228466209

folly/Makefile.am
folly/MicroLock.cpp [new file with mode: 0644]
folly/MicroLock.h [new file with mode: 0644]
folly/MicroSpinLock.h
folly/PicoSpinLock.h
folly/SmallLocks.h
folly/test/SmallLocksBenchmark.cpp [new file with mode: 0644]
folly/test/SmallLocksTest.cpp

index 534ec841e4bd3d8557b472cd4c5cf21e0c685eac..3772e600b2c655352e44d7712b6190b68778a60c 100644 (file)
@@ -257,6 +257,7 @@ nobase_follyinclude_HEADERS = \
        Memory.h \
        MemoryMapping.h \
        MicroSpinLock.h \
+       MicroLock.h \
        MoveWrapper.h \
        MPMCPipeline.h \
        MPMCQueue.h \
@@ -406,6 +407,7 @@ libfolly_la_SOURCES = \
        Random.cpp \
        SafeAssert.cpp \
        SharedMutex.cpp \
+       MicroLock.cpp \
        Singleton.cpp \
        SocketAddress.cpp \
        SpookyHashV1.cpp \
diff --git a/folly/MicroLock.cpp b/folly/MicroLock.cpp
new file mode 100644 (file)
index 0000000..d6656dc
--- /dev/null
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2016 Facebook, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <folly/MicroLock.h>
+
+namespace folly {
+
+void MicroLockCore::lockSlowPath(uint32_t oldWord,
+                                 detail::Futex<>* wordPtr,
+                                 uint32_t slotHeldBit,
+                                 unsigned maxSpins,
+                                 unsigned maxYields) {
+  unsigned newWord;
+  unsigned spins = 0;
+  uint32_t slotWaitBit = slotHeldBit << 1;
+
+retry:
+  if ((oldWord & slotHeldBit) != 0) {
+    ++spins;
+    if (spins > maxSpins + maxYields) {
+      // Somebody appears to have the lock.  Block waiting for the
+      // holder to unlock the lock.  We set heldbit(slot) so that the
+      // lock holder knows to FUTEX_WAKE us.
+      newWord = oldWord | slotWaitBit;
+      if (newWord != oldWord) {
+        if (!wordPtr->compare_exchange_weak(oldWord,
+                                            newWord,
+                                            std::memory_order_relaxed,
+                                            std::memory_order_relaxed)) {
+          goto retry;
+        }
+      }
+      (void)wordPtr->futexWait(newWord, slotHeldBit);
+    } else if (spins > maxSpins) {
+      sched_yield();
+    }
+    oldWord = wordPtr->load(std::memory_order_relaxed);
+    goto retry;
+  }
+
+  newWord = oldWord | slotHeldBit;
+  if (!wordPtr->compare_exchange_weak(oldWord,
+                                      newWord,
+                                      std::memory_order_relaxed,
+                                      std::memory_order_relaxed)) {
+    goto retry;
+  }
+
+  // Locks are traditionally memory barriers, so we emit a full fence
+  // even though we were happy using relaxed atomics for the
+  // lock itself.
+  std::atomic_thread_fence(std::memory_order_seq_cst);
+}
+}
diff --git a/folly/MicroLock.h b/folly/MicroLock.h
new file mode 100644 (file)
index 0000000..d5529af
--- /dev/null
@@ -0,0 +1,208 @@
+/*
+ * Copyright 2016 Facebook, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#pragma once
+
+#include <assert.h>
+#include <limits>
+#include <stdint.h>
+#include <folly/detail/Futex.h>
+#include <folly/Portability.h>
+
+namespace folly {
+
+/**
+ * Tiny exclusive lock that packs four lock slots into a single
+ * byte. Each slot is an independent real, sleeping lock.  The default
+ * lock and unlock functions operate on slot zero, which modifies only
+ * the low two bits of the host byte.
+ *
+ * You should zero-initialize the bits of a MicroLock that you intend
+ * to use.
+ *
+ * If you're not space-constrained, prefer std::mutex, which will
+ * likely be faster, since it has more than two bits of information to
+ * work with.
+ *
+ * You are free to put a MicroLock in a union with some other object.
+ * If, for example, you want to use the bottom two bits of a pointer
+ * as a lock, you can put a MicroLock in a union with the pointer and
+ * limit yourself to MicroLock slot zero, which will use the two
+ * least-significant bits in the bottom byte.
+ *
+ * MicroLock uses a dirty trick: it actually operates on the full
+ * word-size, word-aligned bit of memory into which it is embedded.
+ * It never modifies bits outside the ones it's defined to modify, but
+ * it _accesses_ all the bits in the word for purposes of
+ * futex management.
+ *
+ * The MaxSpins template parameter controls the number of times we
+ * spin trying to acquire the lock.  MaxYields controls the number of
+ * times we call sched_yield; once we've tried to acquire the lock
+ * MaxSpins + MaxYields times, we sleep on the lock futex.
+ * By adjusting these parameters, you can make MicroLock behave as
+ * much or as little like a conventional spinlock as you'd like.
+ *
+ * Performance
+ * -----------
+ *
+ * With the default template options, the timings for uncontended
+ * acquire-then-release come out as follows on Intel(R) Xeon(R) CPU
+ * E5-2660 0 @ 2.20GHz, in @mode/opt, as of the master tree at Tue, 01
+ * Mar 2016 19:48:15.
+ *
+ * ========================================================================
+ * folly/test/SmallLocksBenchmark.cpp          relative  time/iter  iters/s
+ * ========================================================================
+ * MicroSpinLockUncontendedBenchmark                       13.46ns   74.28M
+ * PicoSpinLockUncontendedBenchmark                        14.99ns   66.71M
+ * MicroLockUncontendedBenchmark                           27.06ns   36.96M
+ * StdMutexUncontendedBenchmark                            25.18ns   39.72M
+ * VirtualFunctionCall                                      1.72ns  579.78M
+ * ========================================================================
+ *
+ * (The virtual dispatch benchmark is provided for scale.)
+ *
+ * The contended case for MicroLock is likely to be worse compared to
+ * std::mutex than the contended case is.  Make sure to benchmark your
+ * particular workload.
+ *
+ */
+
+class MicroLockCore {
+ protected:
+  uint8_t lock_;
+  inline detail::Futex<>* word() const;
+  inline uint32_t baseShift(unsigned slot) const;
+  inline uint32_t heldBit(unsigned slot) const;
+  inline uint32_t waitBit(unsigned slot) const;
+  static void lockSlowPath(uint32_t oldWord,
+                           detail::Futex<>* wordPtr,
+                           uint32_t slotHeldBit,
+                           unsigned maxSpins,
+                           unsigned maxYields);
+
+ public:
+  inline void unlock(unsigned slot);
+  inline void unlock() { unlock(0); }
+  inline void init(unsigned slot) { lock_ &= ~(3U << (2 * slot)); }
+  inline void init() { init(0); }
+};
+
+inline detail::Futex<>* MicroLockCore::word() const {
+  uintptr_t lockptr = (uintptr_t)&lock_;
+  lockptr &= ~(sizeof(uint32_t) - 1);
+  return (detail::Futex<>*)lockptr;
+}
+
+inline unsigned MicroLockCore::baseShift(unsigned slot) const {
+  assert(slot < CHAR_BIT / 2);
+  uintptr_t offset_bytes = (uintptr_t)&lock_ - (uintptr_t)word();
+  assert(offset_bytes < sizeof(uint32_t));
+
+  return kIsLittleEndian
+             ? offset_bytes * CHAR_BIT + slot * 2
+             : CHAR_BIT * (sizeof(uint32_t) - offset_bytes - 1) + slot * 2;
+}
+
+inline uint32_t MicroLockCore::heldBit(unsigned slot) const {
+  return 1U << (baseShift(slot) + 0);
+}
+
+inline uint32_t MicroLockCore::waitBit(unsigned slot) const {
+  return 1U << (baseShift(slot) + 1);
+}
+
+void MicroLockCore::unlock(unsigned slot) {
+  detail::Futex<>* wordPtr = word();
+  uint32_t oldWord;
+  uint32_t newWord;
+
+  oldWord = wordPtr->load(std::memory_order_relaxed);
+  do {
+    assert(oldWord & heldBit(slot));
+    newWord = oldWord & ~(heldBit(slot) | waitBit(slot));
+  } while (!wordPtr->compare_exchange_weak(
+      oldWord, newWord, std::memory_order_release, std::memory_order_relaxed));
+
+  if (oldWord & waitBit(slot)) {
+    // We don't track the number of waiters, so wake everyone
+    (void)wordPtr->futexWake(std::numeric_limits<int>::max(), heldBit(slot));
+  }
+}
+
+template <unsigned MaxSpins = 1000, unsigned MaxYields = 0>
+class MicroLockBase : public MicroLockCore {
+ public:
+  inline void lock(unsigned slot);
+  inline void lock() { lock(0); }
+  inline bool try_lock(unsigned slot);
+  inline bool try_lock() { return try_lock(0); }
+};
+
+template <unsigned MaxSpins, unsigned MaxYields>
+bool MicroLockBase<MaxSpins, MaxYields>::try_lock(unsigned slot) {
+
+  // N.B. You might think that try_lock is just the fast path of lock,
+  // but you'd be wrong.  Keep in mind that other parts of our host
+  // word might be changing while we take the lock!  We're not allowed
+  // to fail spuriously if the lock is in fact not held, even if other
+  // people are concurrently modifying other parts of the word.
+  //
+  // We need to loop until we either see firm evidence that somebody
+  // else has the lock (by looking at heldBit) or see our CAS succeed.
+  // A failed CAS by itself does not indicate lock-acquire failure.
+
+  detail::Futex<>* wordPtr = word();
+  uint32_t oldWord = wordPtr->load(std::memory_order_relaxed);
+  do {
+    if (oldWord & heldBit(slot)) {
+      return false;
+    }
+  } while (!wordPtr->compare_exchange_weak(oldWord,
+                                           oldWord | heldBit(slot),
+                                           std::memory_order_acquire,
+                                           std::memory_order_relaxed));
+
+  return true;
+}
+
+template <unsigned MaxSpins, unsigned MaxYields>
+void MicroLockBase<MaxSpins, MaxYields>::lock(unsigned slot) {
+
+  static_assert(MaxSpins + MaxYields < (unsigned)-1, "overflow");
+
+  detail::Futex<>* wordPtr = word();
+  uint32_t oldWord;
+  oldWord = wordPtr->load(std::memory_order_relaxed);
+  if ((oldWord & heldBit(slot)) == 0 &&
+      wordPtr->compare_exchange_weak(oldWord,
+                                     oldWord | heldBit(slot),
+                                     std::memory_order_acquire,
+                                     std::memory_order_relaxed)) {
+    // Fast uncontended case: seq_cst above is our memory barrier
+  } else {
+    // lockSlowPath doesn't have any slot-dependent computation; it
+    // just shifts the input bit.  Make sure its shifting produces the
+    // same result a call to waitBit for our slot would.
+    assert(heldBit(slot) << 1 == waitBit(slot));
+    // lockSlowPath emits its own memory barrier
+    lockSlowPath(oldWord, wordPtr, heldBit(slot), MaxSpins, MaxYields);
+  }
+}
+
+typedef MicroLockBase<> MicroLock;
+}
index 5916dbdc381242842645e531f91e084e0e53b3f3..1318d9e9f0e02d9445d17553cb4e722674eacfec 100644 (file)
@@ -16,7 +16,7 @@
 
 /*
  * N.B. You most likely do _not_ want to use MicroSpinLock or any
- * other kind of spinlock.
+ * other kind of spinlock.  Consider MicroLock instead.
  *
  * In short, spinlocks in preemptive multi-tasking operating systems
  * have serious problems and fast mutexes like std::mutex are almost
index 0a0f28c213bb1dc4b438b6a4174900153c62162e..19472b31fd0d86e913114f03e71fa05777e011f4 100644 (file)
@@ -16,7 +16,7 @@
 
 /*
  * N.B. You most likely do _not_ want to use PicoSpinLock or any other
- * kind of spinlock.
+ * kind of spinlock.  Consider MicroLock instead.
  *
  * In short, spinlocks in preemptive multi-tasking operating systems
  * have serious problems and fast mutexes like std::mutex are almost
index d5fc501a36d25603777ec14061125fd3e7295a1b..b73932b78a094a31bd538ec02dda2a1a7247755c 100644 (file)
@@ -34,6 +34,7 @@
  * @author Jordan DeLong <delong.j@fb.com>
  */
 
+#include <folly/MicroLock.h>
 #include <folly/MicroSpinLock.h>
 #include <folly/PicoSpinLock.h>
 
diff --git a/folly/test/SmallLocksBenchmark.cpp b/folly/test/SmallLocksBenchmark.cpp
new file mode 100644 (file)
index 0000000..19830c0
--- /dev/null
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2016 Facebook, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <folly/Benchmark.h>
+#include <folly/SmallLocks.h>
+
+BENCHMARK(MicroSpinLockUncontendedBenchmark, iters) {
+  folly::MicroSpinLock lock;
+  lock.init();
+  while (iters--) {
+    lock.lock();
+    lock.unlock();
+  }
+}
+
+BENCHMARK(PicoSpinLockUncontendedBenchmark, iters) {
+  // uint8_t would be more fair, but PicoSpinLock needs at lesat two bytes
+  folly::PicoSpinLock<uint16_t> lock;
+  lock.init();
+  while (iters--) {
+    lock.lock();
+    lock.unlock();
+  }
+}
+
+BENCHMARK(MicroLockUncontendedBenchmark, iters) {
+  folly::MicroLock lock;
+  lock.init();
+  while (iters--) {
+    lock.lock();
+    lock.unlock();
+  }
+}
+
+BENCHMARK(StdMutexUncontendedBenchmark, iters) {
+  std::mutex lock;
+  while (iters--) {
+    lock.lock();
+    lock.unlock();
+  }
+}
+
+struct VirtualBase {
+  virtual void foo() = 0;
+  virtual ~VirtualBase() {}
+};
+
+struct VirtualImpl : VirtualBase {
+  virtual void foo() { /* noop */
+  }
+  virtual ~VirtualImpl() {}
+};
+
+#ifndef __clang__
+__attribute__((noinline, noclone)) VirtualBase* makeVirtual() {
+  return new VirtualImpl();
+}
+
+BENCHMARK(VirtualFunctionCall, iters) {
+  VirtualBase* vb = makeVirtual();
+  while (iters--) {
+    vb->foo();
+  }
+  delete vb;
+}
+#endif
+
+int main(int argc, char** argv) {
+  gflags::ParseCommandLineFlags(&argc, &argv, true);
+  folly::runBenchmarks();
+  return 0;
+}
index 509641263a453443240ee22023bbd5b081f9dd9f..d7e27578adca527c95806406b6cf09083916664a 100644 (file)
@@ -21,6 +21,7 @@
 #include <cassert>
 #include <cstdio>
 #include <mutex>
+#include <condition_variable>
 #include <string>
 #include <vector>
 #include <pthread.h>
 
 #include <gtest/gtest.h>
 
-using std::string;
+using folly::MSLGuard;
+using folly::MicroLock;
 using folly::MicroSpinLock;
 using folly::PicoSpinLock;
-using folly::MSLGuard;
+using std::string;
 
 namespace {
 
@@ -166,3 +168,122 @@ TEST(SmallLocks, PicoSpinSigned) {
 TEST(SmallLocks, RegClobber) {
   TestClobber().go();
 }
+
+FOLLY_PACK_PUSH
+static_assert(sizeof(MicroLock) == 1, "Size check failed");
+FOLLY_PACK_POP
+
+namespace {
+
+struct SimpleBarrier {
+
+  SimpleBarrier() : lock_(), cv_(), ready_(false) {}
+
+  void wait() {
+    std::unique_lock<std::mutex> lockHeld(lock_);
+    while (!ready_) {
+      cv_.wait(lockHeld);
+    }
+  }
+
+  void run() {
+    {
+      std::unique_lock<std::mutex> lockHeld(lock_);
+      ready_ = true;
+    }
+
+    cv_.notify_all();
+  }
+
+ private:
+  std::mutex lock_;
+  std::condition_variable cv_;
+  bool ready_;
+};
+}
+
+static void runMicroLockTest() {
+  volatile uint64_t counters[4] = {0, 0, 0, 0};
+  std::vector<std::thread> threads;
+  static const unsigned nrThreads = 20;
+  static const unsigned iterPerThread = 10000;
+  SimpleBarrier startBarrier;
+
+  assert(iterPerThread % 4 == 0);
+
+  // Embed the lock in a larger structure to ensure that we do not
+  // affect bits outside the ones MicroLock is defined to affect.
+  struct {
+    uint8_t a;
+    volatile uint8_t b;
+    union {
+      MicroLock alock;
+      uint8_t c;
+    };
+    volatile uint8_t d;
+  } x;
+
+  uint8_t origB = 'b';
+  uint8_t origD = 'd';
+
+  x.a = 'a';
+  x.b = origB;
+  x.c = 0;
+  x.d = origD;
+
+  // This thread touches other parts of the host word to show that
+  // MicroLock does not interfere with memory outside of the byte
+  // it owns.
+  std::thread adjacentMemoryToucher = std::thread([&] {
+    startBarrier.wait();
+    for (unsigned iter = 0; iter < iterPerThread; ++iter) {
+      if (iter % 2) {
+        x.b++;
+      } else {
+        x.d++;
+      }
+    }
+  });
+
+  for (unsigned i = 0; i < nrThreads; ++i) {
+    threads.emplace_back([&] {
+      startBarrier.wait();
+      for (unsigned iter = 0; iter < iterPerThread; ++iter) {
+        unsigned slotNo = iter % 4;
+        x.alock.lock(slotNo);
+        counters[slotNo] += 1;
+        // The occasional sleep makes it more likely that we'll
+        // exercise the futex-wait path inside MicroLock.
+        if (iter % 1000 == 0) {
+          struct timespec ts = {0, 10000};
+          (void)nanosleep(&ts, nullptr);
+        }
+        x.alock.unlock(slotNo);
+      }
+    });
+  }
+
+  startBarrier.run();
+
+  for (auto it = threads.begin(); it != threads.end(); ++it) {
+    it->join();
+  }
+
+  adjacentMemoryToucher.join();
+
+  EXPECT_EQ(x.a, 'a');
+  EXPECT_EQ(x.b, (uint8_t)(origB + iterPerThread / 2));
+  EXPECT_EQ(x.c, 0);
+  EXPECT_EQ(x.d, (uint8_t)(origD + iterPerThread / 2));
+  for (unsigned i = 0; i < 4; ++i) {
+    EXPECT_EQ(counters[i], ((uint64_t)nrThreads * iterPerThread) / 4);
+  }
+}
+
+TEST(SmallLocks, MicroLock) { runMicroLockTest(); }
+TEST(SmallLocks, MicroLockTryLock) {
+  MicroLock lock;
+  lock.init();
+  EXPECT_TRUE(lock.try_lock());
+  EXPECT_FALSE(lock.try_lock());
+}