Move folly/Baton.h to folly/synchronization/
authorYedidya Feldblum <yfeldblum@fb.com>
Wed, 6 Dec 2017 06:03:40 +0000 (22:03 -0800)
committerFacebook Github Bot <facebook-github-bot@users.noreply.github.com>
Wed, 6 Dec 2017 06:11:17 +0000 (22:11 -0800)
Summary: [Folly] Move `folly/Baton.h` to `folly/synchronization/`.

Reviewed By: phoad, Orvid

Differential Revision: D6490282

fbshipit-source-id: 66e2d25ffe3275d576b97b81c1987709000f6649

39 files changed:
CMakeLists.txt
folly/Baton.h [deleted file]
folly/Makefile.am
folly/Singleton.h
folly/executors/ThreadPoolExecutor.h
folly/executors/task_queue/test/UnboundedBlockingQueueTest.cpp
folly/executors/test/ExecutorTest.cpp
folly/executors/test/SerialExecutorTest.cpp
folly/experimental/flat_combining/FlatCombining.h
folly/experimental/flat_combining/test/FlatCombiningExamples.h
folly/experimental/observer/test/ObserverTest.cpp
folly/experimental/test/FunctionSchedulerTest.cpp
folly/experimental/test/ReadMostlySharedPtrTest.cpp
folly/experimental/test/RefCountTest.cpp
folly/fibers/GenericBaton.h
folly/futures/Future-inl.h
folly/futures/test/Benchmark.cpp
folly/futures/test/FutureTest.cpp
folly/futures/test/InterruptTest.cpp
folly/futures/test/SemiFutureTest.cpp
folly/futures/test/ViaTest.cpp
folly/futures/test/WaitTest.cpp
folly/io/async/EventBase.cpp
folly/io/async/ScopedEventBaseThread.h
folly/io/async/VirtualEventBase.h
folly/io/async/test/EventBaseThreadTest.cpp
folly/io/async/test/NotificationQueueTest.cpp
folly/io/async/test/ScopedEventBaseThreadTest.cpp
folly/synchronization/Baton.h [new file with mode: 0644]
folly/synchronization/LifoSem.h
folly/synchronization/test/BatonBenchmark.cpp [new file with mode: 0644]
folly/synchronization/test/BatonTest.cpp [new file with mode: 0644]
folly/synchronization/test/BatonTestHelpers.h [new file with mode: 0644]
folly/system/test/ThreadNameTest.cpp
folly/test/BatonBenchmark.cpp [deleted file]
folly/test/BatonTest.cpp [deleted file]
folly/test/BatonTestHelpers.h [deleted file]
folly/test/MemoryIdlerTest.cpp
folly/test/ThreadLocalTest.cpp

index ac34e7d4ba2de867c42c911947f25160ee397cd2..fc0084dc13d6ae2fbaceea69975ed522038c3382 100755 (executable)
@@ -537,6 +537,7 @@ if (BUILD_TESTS)
       TEST timeseries_test SOURCES TimeseriesTest.cpp
 
     DIRECTORY synchronization/test/
+      TEST baton_test SOURCES BatonTest.cpp
       TEST call_once_test SOURCES CallOnceTest.cpp
       TEST lifo_sem_test SOURCES LifoSemTests.cpp
 
@@ -558,7 +559,6 @@ if (BUILD_TESTS)
       TEST atomic_linked_list_test SOURCES AtomicLinkedListTest.cpp
       TEST atomic_struct_test SOURCES AtomicStructTest.cpp
       TEST atomic_unordered_map_test SOURCES AtomicUnorderedMapTest.cpp
-      TEST baton_test SOURCES BatonTest.cpp
       TEST bit_iterator_test SOURCES BitIteratorTest.cpp
       TEST bits_test SOURCES BitsTest.cpp
       TEST cacheline_padded_test SOURCES CachelinePaddedTest.cpp
diff --git a/folly/Baton.h b/folly/Baton.h
deleted file mode 100644 (file)
index 8332391..0000000
+++ /dev/null
@@ -1,373 +0,0 @@
-/*
- * Copyright 2017 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 <errno.h>
-#include <stdint.h>
-#include <atomic>
-#include <thread>
-
-#include <folly/detail/Futex.h>
-#include <folly/detail/MemoryIdler.h>
-#include <folly/portability/Asm.h>
-
-namespace folly {
-
-/// A Baton allows a thread to block once and be awoken. The single
-/// poster version (with SinglePoster == true) captures a single
-/// handoff, and during its lifecycle (from construction/reset to
-/// destruction/reset) a baton must either be post()ed and wait()ed
-/// exactly once each, or not at all.
-///
-/// The multi-poster version (SinglePoster == false) allows multiple
-/// concurrent handoff attempts, the first of which completes the
-/// handoff and the rest if any are idempotent.
-///
-/// Baton includes no internal padding, and is only 4 bytes in size.
-/// Any alignment or padding to avoid false sharing is up to the user.
-///
-/// This is basically a stripped-down semaphore that supports (only a
-/// single call to sem_post, when SinglePoster == true) and a single
-/// call to sem_wait.
-///
-/// The non-blocking version (Blocking == false) provides more speed
-/// by using only load acquire and store release operations in the
-/// critical path, at the cost of disallowing blocking and timing out.
-///
-/// The current posix semaphore sem_t isn't too bad, but this provides
-/// more a bit more speed, inlining, smaller size, a guarantee that
-/// the implementation won't change, and compatibility with
-/// DeterministicSchedule.  By having a much more restrictive
-/// lifecycle we can also add a bunch of assertions that can help to
-/// catch race conditions ahead of time.
-template <
-    template <typename> class Atom = std::atomic,
-    bool SinglePoster = true, //  single vs multiple posters
-    bool Blocking = true> // blocking vs spinning
-struct Baton {
-  constexpr Baton() : state_(INIT) {}
-
-  Baton(Baton const&) = delete;
-  Baton& operator=(Baton const&) = delete;
-
-  /// It is an error to destroy a Baton on which a thread is currently
-  /// wait()ing.  In practice this means that the waiter usually takes
-  /// responsibility for destroying the Baton.
-  ~Baton() {
-    // The docblock for this function says that it can't be called when
-    // there is a concurrent waiter.  We assume a strong version of this
-    // requirement in which the caller must _know_ that this is true, they
-    // are not allowed to be merely lucky.  If two threads are involved,
-    // the destroying thread must actually have synchronized with the
-    // waiting thread after wait() returned.  To convey causality the the
-    // waiting thread must have used release semantics and the destroying
-    // thread must have used acquire semantics for that communication,
-    // so we are guaranteed to see the post-wait() value of state_,
-    // which cannot be WAITING.
-    //
-    // Note that since we only care about a single memory location,
-    // the only two plausible memory orders here are relaxed and seq_cst.
-    assert(state_.load(std::memory_order_relaxed) != WAITING);
-  }
-
-  /// Equivalent to destroying the Baton and creating a new one.  It is
-  /// a bug to call this while there is a waiting thread, so in practice
-  /// the waiter will be the one that resets the baton.
-  void reset() {
-    // See ~Baton for a discussion about why relaxed is okay here
-    assert(state_.load(std::memory_order_relaxed) != WAITING);
-
-    // We use a similar argument to justify the use of a relaxed store
-    // here.  Since both wait() and post() are required to be called
-    // only once per lifetime, no thread can actually call those methods
-    // correctly after a reset() unless it synchronizes with the thread
-    // that performed the reset().  If a post() or wait() on another thread
-    // didn't synchronize, then regardless of what operation we performed
-    // here there would be a race on proper use of the Baton's spec
-    // (although not on any particular load and store).  Put another way,
-    // we don't need to synchronize here because anybody that might rely
-    // on such synchronization is required by the baton rules to perform
-    // an additional synchronization that has the desired effect anyway.
-    //
-    // There is actually a similar argument to be made about the
-    // constructor, in which the fenceless constructor initialization
-    // of state_ is piggybacked on whatever synchronization mechanism
-    // distributes knowledge of the Baton's existence
-    state_.store(INIT, std::memory_order_relaxed);
-  }
-
-  /// Causes wait() to wake up.  For each lifetime of a Baton (where a
-  /// lifetime starts at construction or reset() and ends at
-  /// destruction or reset()) there can be at most one call to post(),
-  /// in the single poster version.  Any thread may call post().
-  void post() {
-    if (!Blocking) {
-      /// Non-blocking version
-      ///
-      assert([&] {
-        auto state = state_.load(std::memory_order_relaxed);
-        return (state == INIT || state == EARLY_DELIVERY);
-      }());
-      state_.store(EARLY_DELIVERY, std::memory_order_release);
-      return;
-    }
-
-    /// Blocking versions
-    ///
-    if (SinglePoster) {
-      /// Single poster version
-      ///
-      uint32_t before = state_.load(std::memory_order_acquire);
-
-      assert(before == INIT || before == WAITING || before == TIMED_OUT);
-
-      if (before == INIT &&
-          state_.compare_exchange_strong(before, EARLY_DELIVERY)) {
-        return;
-      }
-
-      assert(before == WAITING || before == TIMED_OUT);
-
-      if (before == TIMED_OUT) {
-        return;
-      }
-
-      assert(before == WAITING);
-      state_.store(LATE_DELIVERY, std::memory_order_release);
-      state_.futexWake(1);
-    } else {
-      /// Multi-poster version
-      ///
-      while (true) {
-        uint32_t before = state_.load(std::memory_order_acquire);
-
-        if (before == INIT &&
-            state_.compare_exchange_strong(before, EARLY_DELIVERY)) {
-          return;
-        }
-
-        if (before == TIMED_OUT) {
-          return;
-        }
-
-        if (before == EARLY_DELIVERY || before == LATE_DELIVERY) {
-          // The reason for not simply returning (without the following
-          // atomic operation) is to avoid the following case:
-          //
-          //  T1:             T2:             T3:
-          //  local1.post();  local2.post();  global.wait();
-          //  global.post();  global.post();  local1.try_wait() == true;
-          //                                  local2.try_wait() == false;
-          //
-          if (state_.fetch_add(0) != before) {
-            continue;
-          }
-          return;
-        }
-
-        assert(before == WAITING);
-        if (!state_.compare_exchange_weak(before, LATE_DELIVERY)) {
-          continue;
-        }
-        state_.futexWake(1);
-        return;
-      }
-    }
-  }
-
-  /// Waits until post() has been called in the current Baton lifetime.
-  /// May be called at most once during a Baton lifetime (construction
-  /// |reset until destruction|reset).  If post is called before wait in
-  /// the current lifetime then this method returns immediately.
-  ///
-  /// The restriction that there can be at most one wait() per lifetime
-  /// could be relaxed somewhat without any perf or size regressions,
-  /// but by making this condition very restrictive we can provide better
-  /// checking in debug builds.
-  void wait() {
-    if (spinWaitForEarlyDelivery()) {
-      assert(state_.load(std::memory_order_acquire) == EARLY_DELIVERY);
-      return;
-    }
-
-    if (!Blocking) {
-      while (!try_wait()) {
-        std::this_thread::yield();
-      }
-      return;
-    }
-
-    // guess we have to block :(
-    uint32_t expected = INIT;
-    if (!state_.compare_exchange_strong(expected, WAITING)) {
-      // CAS failed, last minute reprieve
-      assert(expected == EARLY_DELIVERY);
-      return;
-    }
-
-    while (true) {
-      detail::MemoryIdler::futexWait(state_, WAITING);
-
-      // state_ is the truth even if FUTEX_WAIT reported a matching
-      // FUTEX_WAKE, since we aren't using type-stable storage and we
-      // don't guarantee reuse.  The scenario goes like this: thread
-      // A's last touch of a Baton is a call to wake(), which stores
-      // LATE_DELIVERY and gets an unlucky context switch before delivering
-      // the corresponding futexWake.  Thread B sees LATE_DELIVERY
-      // without consuming a futex event, because it calls futexWait
-      // with an expected value of WAITING and hence doesn't go to sleep.
-      // B returns, so the Baton's memory is reused and becomes another
-      // Baton (or a reuse of this one).  B calls futexWait on the new
-      // Baton lifetime, then A wakes up and delivers a spurious futexWake
-      // to the same memory location.  B's futexWait will then report a
-      // consumed wake event even though state_ is still WAITING.
-      //
-      // It would be possible to add an extra state_ dance to communicate
-      // that the futexWake has been sent so that we can be sure to consume
-      // it before returning, but that would be a perf and complexity hit.
-      uint32_t s = state_.load(std::memory_order_acquire);
-      assert(s == WAITING || s == LATE_DELIVERY);
-
-      if (s == LATE_DELIVERY) {
-        return;
-      }
-      // retry
-    }
-  }
-
-  /// Similar to wait, but with a timeout. The thread is unblocked if the
-  /// timeout expires.
-  /// Note: Only a single call to timed_wait/wait is allowed during a baton's
-  /// life-cycle (from construction/reset to destruction/reset). In other
-  /// words, after timed_wait the caller can't invoke wait/timed_wait/try_wait
-  /// again on the same baton without resetting it.
-  ///
-  /// @param  deadline      Time until which the thread can block
-  /// @return               true if the baton was posted to before timeout,
-  ///                       false otherwise
-  template <typename Clock, typename Duration = typename Clock::duration>
-  bool timed_wait(const std::chrono::time_point<Clock,Duration>& deadline) {
-    static_assert(Blocking, "Non-blocking Baton does not support timed wait.");
-
-    if (spinWaitForEarlyDelivery()) {
-      assert(state_.load(std::memory_order_acquire) == EARLY_DELIVERY);
-      return true;
-    }
-
-    // guess we have to block :(
-    uint32_t expected = INIT;
-    if (!state_.compare_exchange_strong(expected, WAITING)) {
-      // CAS failed, last minute reprieve
-      assert(expected == EARLY_DELIVERY);
-      return true;
-    }
-
-    while (true) {
-      auto rv = state_.futexWaitUntil(WAITING, deadline);
-      if (rv == folly::detail::FutexResult::TIMEDOUT) {
-        state_.store(TIMED_OUT, std::memory_order_release);
-        return false;
-      }
-
-      uint32_t s = state_.load(std::memory_order_acquire);
-      assert(s == WAITING || s == LATE_DELIVERY);
-      if (s == LATE_DELIVERY) {
-        return true;
-      }
-    }
-  }
-
-  /// Similar to timed_wait, but with a duration.
-  template <typename Clock = std::chrono::steady_clock, typename Duration>
-  bool timed_wait(const Duration& duration) {
-    auto deadline = Clock::now() + duration;
-    return timed_wait(deadline);
-  }
-
-  /// Similar to wait, but doesn't block the thread if it hasn't been posted.
-  ///
-  /// try_wait has the following semantics:
-  /// - It is ok to call try_wait any number times on the same baton until
-  ///   try_wait reports that the baton has been posted.
-  /// - It is ok to call timed_wait or wait on the same baton if try_wait
-  ///   reports that baton hasn't been posted.
-  /// - If try_wait indicates that the baton has been posted, it is invalid to
-  ///   call wait, try_wait or timed_wait on the same baton without resetting
-  ///
-  /// @return       true if baton has been posted, false othewise
-  bool try_wait() const {
-    auto s = state_.load(std::memory_order_acquire);
-    assert(s == INIT || s == EARLY_DELIVERY);
-    return s == EARLY_DELIVERY;
-  }
-
- private:
-  enum State : uint32_t {
-    INIT = 0,
-    EARLY_DELIVERY = 1,
-    WAITING = 2,
-    LATE_DELIVERY = 3,
-    TIMED_OUT = 4
-  };
-
-  enum {
-    // Must be positive.  If multiple threads are actively using a
-    // higher-level data structure that uses batons internally, it is
-    // likely that the post() and wait() calls happen almost at the same
-    // time.  In this state, we lose big 50% of the time if the wait goes
-    // to sleep immediately.  On circa-2013 devbox hardware it costs about
-    // 7 usec to FUTEX_WAIT and then be awoken (half the t/iter as the
-    // posix_sem_pingpong test in BatonTests).  We can improve our chances
-    // of EARLY_DELIVERY by spinning for a bit, although we have to balance
-    // this against the loss if we end up sleeping any way.  Spins on this
-    // hw take about 7 nanos (all but 0.5 nanos is the pause instruction).
-    // We give ourself 300 spins, which is about 2 usec of waiting.  As a
-    // partial consolation, since we are using the pause instruction we
-    // are giving a speed boost to the colocated hyperthread.
-    PreBlockAttempts = 300,
-  };
-
-  // Spin for "some time" (see discussion on PreBlockAttempts) waiting
-  // for a post.
-  //
-  // @return       true if we received an early delivery during the wait,
-  //               false otherwise. If the function returns true then
-  //               state_ is guaranteed to be EARLY_DELIVERY
-  bool spinWaitForEarlyDelivery() {
-
-    static_assert(PreBlockAttempts > 0,
-        "isn't this assert clearer than an uninitialized variable warning?");
-    for (int i = 0; i < PreBlockAttempts; ++i) {
-      if (try_wait()) {
-        // hooray!
-        return true;
-      }
-      // The pause instruction is the polite way to spin, but it doesn't
-      // actually affect correctness to omit it if we don't have it.
-      // Pausing donates the full capabilities of the current core to
-      // its other hyperthreads for a dozen cycles or so
-      asm_volatile_pause();
-    }
-
-    return false;
-  }
-
-  detail::Futex<Atom> state_;
-};
-
-} // namespace folly
index 4ca4388d174f41d73d86a03e577b0d28ca8ecffd..985475546e90eb3a5d279c2170e1f06013196c63 100644 (file)
@@ -37,7 +37,6 @@ nobase_follyinclude_HEADERS = \
        AtomicLinkedList.h \
        AtomicStruct.h \
        AtomicUnorderedMap.h \
-       Baton.h \
        Benchmark.h \
        Bits.h \
        CachelinePadded.h \
@@ -433,6 +432,7 @@ nobase_follyinclude_HEADERS = \
        stats/TimeseriesHistogram-defs.h \
        stats/TimeseriesHistogram.h \
        synchronization/AsymmetricMemoryBarrier.h \
+       synchronization/Baton.h \
        synchronization/CallOnce.h \
        synchronization/LifoSem.h \
        synchronization/detail/AtomicUtils.h \
index 01a9d5d12448a4c097664e3d4c7243f47eda87b6..48251b6674bb645a2d147dd2b0bedd3069e768d8 100644 (file)
 // should call reenableInstances.
 
 #pragma once
-#include <folly/Baton.h>
 #include <folly/Demangle.h>
 #include <folly/Exception.h>
 #include <folly/Executor.h>
 #include <folly/detail/StaticSingletonManager.h>
 #include <folly/experimental/ReadMostlySharedPtr.h>
 #include <folly/hash/Hash.h>
+#include <folly/synchronization/Baton.h>
 
 #include <algorithm>
 #include <atomic>
index a92726ef51c0e0451f5d0fa68b8c448bee9d31ec..0e5bcc178a3cd5c97b90c8a7791b12cd53a02809 100644 (file)
@@ -15,7 +15,6 @@
  */
 
 #pragma once
-#include <folly/Baton.h>
 #include <folly/Executor.h>
 #include <folly/Memory.h>
 #include <folly/RWSpinLock.h>
@@ -23,6 +22,7 @@
 #include <folly/executors/task_queue/LifoSemMPMCQueue.h>
 #include <folly/executors/thread_factory/NamedThreadFactory.h>
 #include <folly/io/async/Request.h>
+#include <folly/synchronization/Baton.h>
 
 #include <algorithm>
 #include <mutex>
index a7b1efd510efe4fd72fa27d9516c2b6ef0e4f86f..c3e7d0d40f1782964b158fcc53fe1434bf8c6ad1 100644 (file)
@@ -14,8 +14,8 @@
  * limitations under the License.
  */
 #include <folly/executors/task_queue/UnboundedBlockingQueue.h>
-#include <folly/Baton.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 #include <thread>
 
 using namespace folly;
index 2bbc56a1fa270081492f2efa545b7b75ba3959c2..e017ca789122ce3c964ec6ab347965c691e5a1ab 100644 (file)
  * limitations under the License.
  */
 
-#include <folly/Baton.h>
 #include <folly/executors/InlineExecutor.h>
 #include <folly/executors/ManualExecutor.h>
 #include <folly/executors/QueuedImmediateExecutor.h>
 #include <folly/futures/Future.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 
 // TODO(jsedgwick) move this test to executors/test/ once the tested executors
 // have all moved
index ae487591f860c80ee5d6275ec789535ba7c590ff..5e022a1f72c903b61ab05b62f2d0f1e6287176bd 100644 (file)
 
 #include <chrono>
 
-#include <folly/Baton.h>
 #include <folly/executors/CPUThreadPoolExecutor.h>
 #include <folly/executors/InlineExecutor.h>
 #include <folly/executors/SerialExecutor.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 
 using namespace std::chrono;
 using folly::SerialExecutor;
index 63fe1ce7bc2fe82af8fb5284cefd1860161a88a9..235e24b29f01f94a6a0404443506a5f431d97107 100644 (file)
 
 #pragma once
 
-#include <folly/Baton.h>
 #include <folly/Function.h>
 #include <folly/IndexedMemPool.h>
 #include <folly/Portability.h>
 #include <folly/concurrency/CacheLocality.h>
+#include <folly/synchronization/Baton.h>
 
 #include <atomic>
 #include <cassert>
index 4736dfa82b52a70c926a67de6fa375af2c93a7e3..fe6ab184ccde5c1abc081bdd2e6d065ac3d76167 100644 (file)
@@ -20,8 +20,8 @@
 #include <memory>
 #include <mutex>
 
-#include <folly/Baton.h>
 #include <folly/experimental/flat_combining/FlatCombining.h>
+#include <folly/synchronization/Baton.h>
 
 namespace folly {
 
index ed372f5aadd2b63354b09a44d05ab48ae2d64834..1c3bb28f7bf3f45c43a51cea1a228157bde01fc1 100644 (file)
@@ -16,9 +16,9 @@
 
 #include <thread>
 
-#include <folly/Baton.h>
 #include <folly/experimental/observer/SimpleObservable.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 
 using namespace folly::observer;
 
index ec94395851bd52e38f99b69c4a920eae960f4bf6..b869f2a856eea59515768fa6086819f4061b5d2b 100644 (file)
 
 #include <boost/thread.hpp>
 
-#include <folly/Baton.h>
 #include <folly/Random.h>
 #include <folly/experimental/FunctionScheduler.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 
 #if defined(__linux__)
 #include <dlfcn.h>
index 089961159c206977994f0b40f51ef01f0fe3fd2e..789eeecd9c43c3eae4849815d4451381a17f60d2 100644 (file)
 #include <mutex>
 #include <thread>
 
-#include <folly/Baton.h>
 #include <folly/Memory.h>
 #include <folly/experimental/RCURefCount.h>
 #include <folly/experimental/ReadMostlySharedPtr.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 
 using folly::ReadMostlyMainPtr;
 using folly::ReadMostlyWeakPtr;
index a63de14d26414cb85db4cef938158b3a11b60148..3c75ae3cf0dc67616d1b44839993f58c63f73be1 100644 (file)
  */
 #include <thread>
 
-#include <folly/Baton.h>
 #include <folly/experimental/RCURefCount.h>
 #include <folly/experimental/TLRefCount.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 
 namespace folly {
 
index 93dbebaea6d331ed7ab87e38ca0bd0d1f0c4e9e2..6cab1bf0e3fed834d2456f573d27a62d01fafd6c 100644 (file)
@@ -15,7 +15,7 @@
  */
 #pragma once
 
-#include <folly/Baton.h>
+#include <folly/synchronization/Baton.h>
 
 #include <folly/fibers/Baton.h>
 
index 5818244cdc3443818ec8d887815eb1c8aef9c2dd..be53ff7c9a0721d2ece56f899e65c2edda82147d 100644 (file)
 #include <chrono>
 #include <thread>
 
-#include <folly/Baton.h>
 #include <folly/Optional.h>
 #include <folly/executors/InlineExecutor.h>
 #include <folly/futures/Timekeeper.h>
 #include <folly/futures/detail/Core.h>
+#include <folly/synchronization/Baton.h>
 
 #ifndef FOLLY_FUTURE_USING_FIBER
 #if FOLLY_MOBILE || defined(__APPLE__)
index a9c743de70a4f089994718e876563a1b731f426e..336ce7f7894b881131946ce3bef746e8caf340c3 100644 (file)
  */
 
 #include <folly/Benchmark.h>
-#include <folly/Baton.h>
 #include <folly/executors/InlineExecutor.h>
 #include <folly/futures/Future.h>
 #include <folly/futures/Promise.h>
 #include <folly/portability/GFlags.h>
 #include <folly/portability/Semaphore.h>
+#include <folly/synchronization/Baton.h>
 
 #include <vector>
 
index e269aa153b7329ad0df2530f4beceed13d6f890b..ae321fde4ca20253e8aad642c96ff6035f5ffabe 100644 (file)
  * limitations under the License.
  */
 
-#include <folly/Baton.h>
+#include <folly/futures/Future.h>
 #include <folly/Executor.h>
 #include <folly/Memory.h>
 #include <folly/Unit.h>
 #include <folly/dynamic.h>
-#include <folly/futures/Future.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 
 #include <algorithm>
 #include <atomic>
index 7c51bb1e98af512a402bbb0ac5326c065bd62675..fc87606b9146c8c0080a345666d2044367e0df65 100644 (file)
  * limitations under the License.
  */
 
-#include <folly/Baton.h>
 #include <folly/futures/Future.h>
 #include <folly/futures/Promise.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 
 using namespace folly;
 
index 39ea108b31a6d68437fd9588792857ebb835ab35..7b927021e7dce159d36982857e4b0ccf1a53cb97 100644 (file)
@@ -14,7 +14,6 @@
  * limitations under the License.
  */
 
-#include <folly/Baton.h>
 #include <folly/Executor.h>
 #include <folly/Memory.h>
 #include <folly/Unit.h>
@@ -22,6 +21,7 @@
 #include <folly/futures/Future.h>
 #include <folly/io/async/EventBase.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 
 #include <algorithm>
 #include <atomic>
index fb2008a8d0236f65472af5f78849eedf31328aae..f7a81c9956bb4687b50f83e351a513ce9160ce6d 100644 (file)
 
 #include <thread>
 
-#include <folly/Baton.h>
 #include <folly/MPMCQueue.h>
 #include <folly/executors/DrivableExecutor.h>
 #include <folly/executors/InlineExecutor.h>
 #include <folly/executors/ManualExecutor.h>
 #include <folly/futures/Future.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 
 using namespace folly;
 
index 57e4e172d2f9354e2be292d0f9dc982e23e30fac..1d1b287d8f087d8202ddcf984912b8a5cadab43d 100644 (file)
 
 #include <queue>
 
-#include <folly/Baton.h>
 #include <folly/futures/Future.h>
 #include <folly/io/async/EventBase.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 
 using namespace folly;
 using std::vector;
index c1ed73365413b866f367d2b4d5504012cb6dfb72..1466d72159f3ac1d74939dd0d9fef46889beaed3 100644 (file)
 #include <mutex>
 #include <thread>
 
-#include <folly/Baton.h>
 #include <folly/Memory.h>
 #include <folly/io/async/NotificationQueue.h>
 #include <folly/io/async/VirtualEventBase.h>
 #include <folly/portability/Unistd.h>
+#include <folly/synchronization/Baton.h>
 #include <folly/system/ThreadName.h>
 
 namespace folly {
index c81ab3930410b0fb9b53fc891fb9e4320a657745..c234dc0046be5a5fc69f7e34d22d478d2e8daaaf 100644 (file)
@@ -19,8 +19,8 @@
 #include <memory>
 #include <thread>
 
-#include <folly/Baton.h>
 #include <folly/io/async/EventBase.h>
+#include <folly/synchronization/Baton.h>
 
 namespace folly {
 
index 5120f21efea1e085a1fc8f0fb4ebb135d46bd198..30e8404b1e1723c0bca437900920bff346094f0f 100644 (file)
@@ -18,9 +18,9 @@
 
 #include <future>
 
-#include <folly/Baton.h>
 #include <folly/Executor.h>
 #include <folly/io/async/EventBase.h>
+#include <folly/synchronization/Baton.h>
 
 namespace folly {
 
index e804501e7368f9c53b6737677129d6861f8a6804..88ed75b26755a039a5e902824b921aa18c7f4fe1 100644 (file)
@@ -18,9 +18,9 @@
 
 #include <chrono>
 
-#include <folly/Baton.h>
 #include <folly/io/async/EventBaseManager.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 #include <folly/system/ThreadName.h>
 
 using namespace std;
index 727eaad2e7dd34e9eaf77ff3de7b90a074a58638..2ff48d542cee5aab84b738459a4aa4a19a147c4b 100644 (file)
@@ -22,9 +22,9 @@
 #include <list>
 #include <thread>
 
-#include <folly/Baton.h>
 #include <folly/io/async/ScopedEventBaseThread.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 
 #ifndef _WIN32
 #include <sys/wait.h>
index 6f1d226672cb037169fd122cf5e033ee8fcc929e..321a088876133fa3c8c5e073a7af5e86b811a36b 100644 (file)
 #include <chrono>
 #include <string>
 
-#include <folly/Baton.h>
 #include <folly/Optional.h>
 #include <folly/io/async/EventBaseManager.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 #include <folly/system/ThreadName.h>
 
 using namespace std;
diff --git a/folly/synchronization/Baton.h b/folly/synchronization/Baton.h
new file mode 100644 (file)
index 0000000..8332391
--- /dev/null
@@ -0,0 +1,373 @@
+/*
+ * Copyright 2017 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 <errno.h>
+#include <stdint.h>
+#include <atomic>
+#include <thread>
+
+#include <folly/detail/Futex.h>
+#include <folly/detail/MemoryIdler.h>
+#include <folly/portability/Asm.h>
+
+namespace folly {
+
+/// A Baton allows a thread to block once and be awoken. The single
+/// poster version (with SinglePoster == true) captures a single
+/// handoff, and during its lifecycle (from construction/reset to
+/// destruction/reset) a baton must either be post()ed and wait()ed
+/// exactly once each, or not at all.
+///
+/// The multi-poster version (SinglePoster == false) allows multiple
+/// concurrent handoff attempts, the first of which completes the
+/// handoff and the rest if any are idempotent.
+///
+/// Baton includes no internal padding, and is only 4 bytes in size.
+/// Any alignment or padding to avoid false sharing is up to the user.
+///
+/// This is basically a stripped-down semaphore that supports (only a
+/// single call to sem_post, when SinglePoster == true) and a single
+/// call to sem_wait.
+///
+/// The non-blocking version (Blocking == false) provides more speed
+/// by using only load acquire and store release operations in the
+/// critical path, at the cost of disallowing blocking and timing out.
+///
+/// The current posix semaphore sem_t isn't too bad, but this provides
+/// more a bit more speed, inlining, smaller size, a guarantee that
+/// the implementation won't change, and compatibility with
+/// DeterministicSchedule.  By having a much more restrictive
+/// lifecycle we can also add a bunch of assertions that can help to
+/// catch race conditions ahead of time.
+template <
+    template <typename> class Atom = std::atomic,
+    bool SinglePoster = true, //  single vs multiple posters
+    bool Blocking = true> // blocking vs spinning
+struct Baton {
+  constexpr Baton() : state_(INIT) {}
+
+  Baton(Baton const&) = delete;
+  Baton& operator=(Baton const&) = delete;
+
+  /// It is an error to destroy a Baton on which a thread is currently
+  /// wait()ing.  In practice this means that the waiter usually takes
+  /// responsibility for destroying the Baton.
+  ~Baton() {
+    // The docblock for this function says that it can't be called when
+    // there is a concurrent waiter.  We assume a strong version of this
+    // requirement in which the caller must _know_ that this is true, they
+    // are not allowed to be merely lucky.  If two threads are involved,
+    // the destroying thread must actually have synchronized with the
+    // waiting thread after wait() returned.  To convey causality the the
+    // waiting thread must have used release semantics and the destroying
+    // thread must have used acquire semantics for that communication,
+    // so we are guaranteed to see the post-wait() value of state_,
+    // which cannot be WAITING.
+    //
+    // Note that since we only care about a single memory location,
+    // the only two plausible memory orders here are relaxed and seq_cst.
+    assert(state_.load(std::memory_order_relaxed) != WAITING);
+  }
+
+  /// Equivalent to destroying the Baton and creating a new one.  It is
+  /// a bug to call this while there is a waiting thread, so in practice
+  /// the waiter will be the one that resets the baton.
+  void reset() {
+    // See ~Baton for a discussion about why relaxed is okay here
+    assert(state_.load(std::memory_order_relaxed) != WAITING);
+
+    // We use a similar argument to justify the use of a relaxed store
+    // here.  Since both wait() and post() are required to be called
+    // only once per lifetime, no thread can actually call those methods
+    // correctly after a reset() unless it synchronizes with the thread
+    // that performed the reset().  If a post() or wait() on another thread
+    // didn't synchronize, then regardless of what operation we performed
+    // here there would be a race on proper use of the Baton's spec
+    // (although not on any particular load and store).  Put another way,
+    // we don't need to synchronize here because anybody that might rely
+    // on such synchronization is required by the baton rules to perform
+    // an additional synchronization that has the desired effect anyway.
+    //
+    // There is actually a similar argument to be made about the
+    // constructor, in which the fenceless constructor initialization
+    // of state_ is piggybacked on whatever synchronization mechanism
+    // distributes knowledge of the Baton's existence
+    state_.store(INIT, std::memory_order_relaxed);
+  }
+
+  /// Causes wait() to wake up.  For each lifetime of a Baton (where a
+  /// lifetime starts at construction or reset() and ends at
+  /// destruction or reset()) there can be at most one call to post(),
+  /// in the single poster version.  Any thread may call post().
+  void post() {
+    if (!Blocking) {
+      /// Non-blocking version
+      ///
+      assert([&] {
+        auto state = state_.load(std::memory_order_relaxed);
+        return (state == INIT || state == EARLY_DELIVERY);
+      }());
+      state_.store(EARLY_DELIVERY, std::memory_order_release);
+      return;
+    }
+
+    /// Blocking versions
+    ///
+    if (SinglePoster) {
+      /// Single poster version
+      ///
+      uint32_t before = state_.load(std::memory_order_acquire);
+
+      assert(before == INIT || before == WAITING || before == TIMED_OUT);
+
+      if (before == INIT &&
+          state_.compare_exchange_strong(before, EARLY_DELIVERY)) {
+        return;
+      }
+
+      assert(before == WAITING || before == TIMED_OUT);
+
+      if (before == TIMED_OUT) {
+        return;
+      }
+
+      assert(before == WAITING);
+      state_.store(LATE_DELIVERY, std::memory_order_release);
+      state_.futexWake(1);
+    } else {
+      /// Multi-poster version
+      ///
+      while (true) {
+        uint32_t before = state_.load(std::memory_order_acquire);
+
+        if (before == INIT &&
+            state_.compare_exchange_strong(before, EARLY_DELIVERY)) {
+          return;
+        }
+
+        if (before == TIMED_OUT) {
+          return;
+        }
+
+        if (before == EARLY_DELIVERY || before == LATE_DELIVERY) {
+          // The reason for not simply returning (without the following
+          // atomic operation) is to avoid the following case:
+          //
+          //  T1:             T2:             T3:
+          //  local1.post();  local2.post();  global.wait();
+          //  global.post();  global.post();  local1.try_wait() == true;
+          //                                  local2.try_wait() == false;
+          //
+          if (state_.fetch_add(0) != before) {
+            continue;
+          }
+          return;
+        }
+
+        assert(before == WAITING);
+        if (!state_.compare_exchange_weak(before, LATE_DELIVERY)) {
+          continue;
+        }
+        state_.futexWake(1);
+        return;
+      }
+    }
+  }
+
+  /// Waits until post() has been called in the current Baton lifetime.
+  /// May be called at most once during a Baton lifetime (construction
+  /// |reset until destruction|reset).  If post is called before wait in
+  /// the current lifetime then this method returns immediately.
+  ///
+  /// The restriction that there can be at most one wait() per lifetime
+  /// could be relaxed somewhat without any perf or size regressions,
+  /// but by making this condition very restrictive we can provide better
+  /// checking in debug builds.
+  void wait() {
+    if (spinWaitForEarlyDelivery()) {
+      assert(state_.load(std::memory_order_acquire) == EARLY_DELIVERY);
+      return;
+    }
+
+    if (!Blocking) {
+      while (!try_wait()) {
+        std::this_thread::yield();
+      }
+      return;
+    }
+
+    // guess we have to block :(
+    uint32_t expected = INIT;
+    if (!state_.compare_exchange_strong(expected, WAITING)) {
+      // CAS failed, last minute reprieve
+      assert(expected == EARLY_DELIVERY);
+      return;
+    }
+
+    while (true) {
+      detail::MemoryIdler::futexWait(state_, WAITING);
+
+      // state_ is the truth even if FUTEX_WAIT reported a matching
+      // FUTEX_WAKE, since we aren't using type-stable storage and we
+      // don't guarantee reuse.  The scenario goes like this: thread
+      // A's last touch of a Baton is a call to wake(), which stores
+      // LATE_DELIVERY and gets an unlucky context switch before delivering
+      // the corresponding futexWake.  Thread B sees LATE_DELIVERY
+      // without consuming a futex event, because it calls futexWait
+      // with an expected value of WAITING and hence doesn't go to sleep.
+      // B returns, so the Baton's memory is reused and becomes another
+      // Baton (or a reuse of this one).  B calls futexWait on the new
+      // Baton lifetime, then A wakes up and delivers a spurious futexWake
+      // to the same memory location.  B's futexWait will then report a
+      // consumed wake event even though state_ is still WAITING.
+      //
+      // It would be possible to add an extra state_ dance to communicate
+      // that the futexWake has been sent so that we can be sure to consume
+      // it before returning, but that would be a perf and complexity hit.
+      uint32_t s = state_.load(std::memory_order_acquire);
+      assert(s == WAITING || s == LATE_DELIVERY);
+
+      if (s == LATE_DELIVERY) {
+        return;
+      }
+      // retry
+    }
+  }
+
+  /// Similar to wait, but with a timeout. The thread is unblocked if the
+  /// timeout expires.
+  /// Note: Only a single call to timed_wait/wait is allowed during a baton's
+  /// life-cycle (from construction/reset to destruction/reset). In other
+  /// words, after timed_wait the caller can't invoke wait/timed_wait/try_wait
+  /// again on the same baton without resetting it.
+  ///
+  /// @param  deadline      Time until which the thread can block
+  /// @return               true if the baton was posted to before timeout,
+  ///                       false otherwise
+  template <typename Clock, typename Duration = typename Clock::duration>
+  bool timed_wait(const std::chrono::time_point<Clock,Duration>& deadline) {
+    static_assert(Blocking, "Non-blocking Baton does not support timed wait.");
+
+    if (spinWaitForEarlyDelivery()) {
+      assert(state_.load(std::memory_order_acquire) == EARLY_DELIVERY);
+      return true;
+    }
+
+    // guess we have to block :(
+    uint32_t expected = INIT;
+    if (!state_.compare_exchange_strong(expected, WAITING)) {
+      // CAS failed, last minute reprieve
+      assert(expected == EARLY_DELIVERY);
+      return true;
+    }
+
+    while (true) {
+      auto rv = state_.futexWaitUntil(WAITING, deadline);
+      if (rv == folly::detail::FutexResult::TIMEDOUT) {
+        state_.store(TIMED_OUT, std::memory_order_release);
+        return false;
+      }
+
+      uint32_t s = state_.load(std::memory_order_acquire);
+      assert(s == WAITING || s == LATE_DELIVERY);
+      if (s == LATE_DELIVERY) {
+        return true;
+      }
+    }
+  }
+
+  /// Similar to timed_wait, but with a duration.
+  template <typename Clock = std::chrono::steady_clock, typename Duration>
+  bool timed_wait(const Duration& duration) {
+    auto deadline = Clock::now() + duration;
+    return timed_wait(deadline);
+  }
+
+  /// Similar to wait, but doesn't block the thread if it hasn't been posted.
+  ///
+  /// try_wait has the following semantics:
+  /// - It is ok to call try_wait any number times on the same baton until
+  ///   try_wait reports that the baton has been posted.
+  /// - It is ok to call timed_wait or wait on the same baton if try_wait
+  ///   reports that baton hasn't been posted.
+  /// - If try_wait indicates that the baton has been posted, it is invalid to
+  ///   call wait, try_wait or timed_wait on the same baton without resetting
+  ///
+  /// @return       true if baton has been posted, false othewise
+  bool try_wait() const {
+    auto s = state_.load(std::memory_order_acquire);
+    assert(s == INIT || s == EARLY_DELIVERY);
+    return s == EARLY_DELIVERY;
+  }
+
+ private:
+  enum State : uint32_t {
+    INIT = 0,
+    EARLY_DELIVERY = 1,
+    WAITING = 2,
+    LATE_DELIVERY = 3,
+    TIMED_OUT = 4
+  };
+
+  enum {
+    // Must be positive.  If multiple threads are actively using a
+    // higher-level data structure that uses batons internally, it is
+    // likely that the post() and wait() calls happen almost at the same
+    // time.  In this state, we lose big 50% of the time if the wait goes
+    // to sleep immediately.  On circa-2013 devbox hardware it costs about
+    // 7 usec to FUTEX_WAIT and then be awoken (half the t/iter as the
+    // posix_sem_pingpong test in BatonTests).  We can improve our chances
+    // of EARLY_DELIVERY by spinning for a bit, although we have to balance
+    // this against the loss if we end up sleeping any way.  Spins on this
+    // hw take about 7 nanos (all but 0.5 nanos is the pause instruction).
+    // We give ourself 300 spins, which is about 2 usec of waiting.  As a
+    // partial consolation, since we are using the pause instruction we
+    // are giving a speed boost to the colocated hyperthread.
+    PreBlockAttempts = 300,
+  };
+
+  // Spin for "some time" (see discussion on PreBlockAttempts) waiting
+  // for a post.
+  //
+  // @return       true if we received an early delivery during the wait,
+  //               false otherwise. If the function returns true then
+  //               state_ is guaranteed to be EARLY_DELIVERY
+  bool spinWaitForEarlyDelivery() {
+
+    static_assert(PreBlockAttempts > 0,
+        "isn't this assert clearer than an uninitialized variable warning?");
+    for (int i = 0; i < PreBlockAttempts; ++i) {
+      if (try_wait()) {
+        // hooray!
+        return true;
+      }
+      // The pause instruction is the polite way to spin, but it doesn't
+      // actually affect correctness to omit it if we don't have it.
+      // Pausing donates the full capabilities of the current core to
+      // its other hyperthreads for a dozen cycles or so
+      asm_volatile_pause();
+    }
+
+    return false;
+  }
+
+  detail::Futex<Atom> state_;
+};
+
+} // namespace folly
index 90176094a857c6d0e573a1f0dee46dcd2e7573c2..ef27df4ee1d43490a377c5ea12735a2190aa5947 100644 (file)
 #include <system_error>
 
 #include <folly/AtomicStruct.h>
-#include <folly/Baton.h>
 #include <folly/CachelinePadded.h>
 #include <folly/IndexedMemPool.h>
 #include <folly/Likely.h>
+#include <folly/synchronization/Baton.h>
 
 namespace folly {
 
diff --git a/folly/synchronization/test/BatonBenchmark.cpp b/folly/synchronization/test/BatonBenchmark.cpp
new file mode 100644 (file)
index 0000000..2512628
--- /dev/null
@@ -0,0 +1,104 @@
+/*
+ * Copyright 2017 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/synchronization/Baton.h>
+
+#include <thread>
+
+#include <folly/Benchmark.h>
+#include <folly/portability/GFlags.h>
+#include <folly/portability/GTest.h>
+#include <folly/portability/Semaphore.h>
+#include <folly/synchronization/test/BatonTestHelpers.h>
+#include <folly/test/DeterministicSchedule.h>
+
+using namespace folly;
+using namespace folly::test;
+using folly::detail::EmulatedFutexAtomic;
+
+typedef DeterministicSchedule DSched;
+
+BENCHMARK(baton_pingpong_single_poster_blocking, iters) {
+  run_pingpong_test<std::atomic, true, true>(iters);
+}
+
+BENCHMARK(baton_pingpong_multi_poster_blocking, iters) {
+  run_pingpong_test<std::atomic, false, true>(iters);
+}
+
+BENCHMARK(baton_pingpong_single_poster_nonblocking, iters) {
+  run_pingpong_test<std::atomic, true, false>(iters);
+}
+
+BENCHMARK(baton_pingpong_multi_poster_nonblocking, iters) {
+  run_pingpong_test<std::atomic, false, false>(iters);
+}
+
+BENCHMARK_DRAW_LINE()
+
+BENCHMARK(baton_pingpong_emulated_futex_single_poster_blocking, iters) {
+  run_pingpong_test<EmulatedFutexAtomic, true, true>(iters);
+}
+
+BENCHMARK(baton_pingpong_emulated_futex_multi_poster_blocking, iters) {
+  run_pingpong_test<EmulatedFutexAtomic, false, true>(iters);
+}
+
+BENCHMARK(baton_pingpong_emulated_futex_single_poster_nonblocking, iters) {
+  run_pingpong_test<EmulatedFutexAtomic, true, false>(iters);
+}
+
+BENCHMARK(baton_pingpong_emulated_futex_multi_poster_nonblocking, iters) {
+  run_pingpong_test<EmulatedFutexAtomic, false, false>(iters);
+}
+
+BENCHMARK_DRAW_LINE()
+
+BENCHMARK(posix_sem_pingpong, iters) {
+  sem_t sems[3];
+  sem_t* a = sems + 0;
+  sem_t* b = sems + 2; // to get it on a different cache line
+
+  sem_init(a, 0, 0);
+  sem_init(b, 0, 0);
+  auto thr = std::thread([=] {
+    for (size_t i = 0; i < iters; ++i) {
+      sem_wait(a);
+      sem_post(b);
+    }
+  });
+  for (size_t i = 0; i < iters; ++i) {
+    sem_post(a);
+    sem_wait(b);
+  }
+  thr.join();
+}
+
+// I am omitting a benchmark result snapshot because these microbenchmarks
+// mainly illustrate that PreBlockAttempts is very effective for rapid
+// handoffs.  The performance of Baton and sem_t is essentially identical
+// to the required futex calls for the blocking case
+
+int main(int argc, char** argv) {
+  testing::InitGoogleTest(&argc, argv);
+  gflags::ParseCommandLineFlags(&argc, &argv, true);
+
+  auto rv = RUN_ALL_TESTS();
+  if (!rv && FLAGS_benchmark) {
+    folly::runBenchmarks();
+  }
+  return rv;
+}
diff --git a/folly/synchronization/test/BatonTest.cpp b/folly/synchronization/test/BatonTest.cpp
new file mode 100644 (file)
index 0000000..2e6f516
--- /dev/null
@@ -0,0 +1,285 @@
+/*
+ * Copyright 2017 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/synchronization/Baton.h>
+
+#include <thread>
+
+#include <folly/portability/GTest.h>
+#include <folly/synchronization/test/BatonTestHelpers.h>
+#include <folly/test/DeterministicSchedule.h>
+
+using namespace folly;
+using namespace folly::test;
+using folly::detail::EmulatedFutexAtomic;
+
+/// Basic test
+
+TEST(Baton, basic_single_poster_blocking) {
+  run_basic_test<std::atomic, true, true>();
+  run_basic_test<EmulatedFutexAtomic, true, true>();
+  run_basic_test<DeterministicAtomic, true, true>();
+}
+
+TEST(Baton, basic_single_poster_nonblocking) {
+  run_basic_test<std::atomic, true, false>();
+  run_basic_test<EmulatedFutexAtomic, true, false>();
+  run_basic_test<DeterministicAtomic, true, false>();
+}
+
+TEST(Baton, basic_multi_poster_blocking) {
+  run_basic_test<std::atomic, false, true>();
+}
+
+TEST(Baton, basic_multi_poster_nonblocking) {
+  run_basic_test<std::atomic, false, false>();
+}
+
+/// Ping pong tests
+
+TEST(Baton, pingpong_single_poster_blocking) {
+  DSched sched(DSched::uniform(0));
+
+  run_pingpong_test<DeterministicAtomic, true, true>(1000);
+}
+
+TEST(Baton, pingpong_single_poster_nonblocking) {
+  DSched sched(DSched::uniform(0));
+
+  run_pingpong_test<DeterministicAtomic, true, false>(1000);
+}
+
+TEST(Baton, pingpong_multi_poster_blocking) {
+  DSched sched(DSched::uniform(0));
+
+  run_pingpong_test<DeterministicAtomic, false, true>(1000);
+}
+
+TEST(Baton, pingpong_multi_poster_nonblocking) {
+  DSched sched(DSched::uniform(0));
+
+  run_pingpong_test<DeterministicAtomic, false, false>(1000);
+}
+
+/// Timed wait tests - Nonblocking Baton does not support timed_wait()
+
+// Timed wait basic system clock tests
+
+TEST(Baton, timed_wait_basic_system_clock_single_poster) {
+  run_basic_timed_wait_tests<std::atomic, std::chrono::system_clock, true>();
+  run_basic_timed_wait_tests<
+      EmulatedFutexAtomic,
+      std::chrono::system_clock,
+      true>();
+  run_basic_timed_wait_tests<
+      DeterministicAtomic,
+      std::chrono::system_clock,
+      true>();
+}
+
+TEST(Baton, timed_wait_basic_system_clock_multi_poster) {
+  run_basic_timed_wait_tests<std::atomic, std::chrono::system_clock, false>();
+  run_basic_timed_wait_tests<
+      EmulatedFutexAtomic,
+      std::chrono::system_clock,
+      false>();
+  run_basic_timed_wait_tests<
+      DeterministicAtomic,
+      std::chrono::system_clock,
+      false>();
+}
+
+// Timed wait timeout system clock tests
+
+TEST(Baton, timed_wait_timeout_system_clock_single_poster) {
+  run_timed_wait_tmo_tests<std::atomic, std::chrono::system_clock, true>();
+  run_timed_wait_tmo_tests<
+      EmulatedFutexAtomic,
+      std::chrono::system_clock,
+      true>();
+  run_timed_wait_tmo_tests<
+      DeterministicAtomic,
+      std::chrono::system_clock,
+      true>();
+}
+
+TEST(Baton, timed_wait_timeout_system_clock_multi_poster) {
+  run_timed_wait_tmo_tests<std::atomic, std::chrono::system_clock, false>();
+  run_timed_wait_tmo_tests<
+      EmulatedFutexAtomic,
+      std::chrono::system_clock,
+      false>();
+  run_timed_wait_tmo_tests<
+      DeterministicAtomic,
+      std::chrono::system_clock,
+      false>();
+}
+
+// Timed wait regular system clock tests
+
+TEST(Baton, timed_wait_system_clock_single_poster) {
+  run_timed_wait_regular_test<std::atomic, std::chrono::system_clock, true>();
+  run_timed_wait_regular_test<
+      EmulatedFutexAtomic,
+      std::chrono::system_clock,
+      true>();
+  run_timed_wait_regular_test<
+      DeterministicAtomic,
+      std::chrono::system_clock,
+      true>();
+}
+
+TEST(Baton, timed_wait_system_clock_multi_poster) {
+  run_timed_wait_regular_test<std::atomic, std::chrono::system_clock, false>();
+  run_timed_wait_regular_test<
+      EmulatedFutexAtomic,
+      std::chrono::system_clock,
+      false>();
+  run_timed_wait_regular_test<
+      DeterministicAtomic,
+      std::chrono::system_clock,
+      false>();
+}
+
+// Timed wait basic steady clock tests
+
+TEST(Baton, timed_wait_basic_steady_clock_single_poster) {
+  run_basic_timed_wait_tests<std::atomic, std::chrono::steady_clock, true>();
+  run_basic_timed_wait_tests<
+      EmulatedFutexAtomic,
+      std::chrono::steady_clock,
+      true>();
+  run_basic_timed_wait_tests<
+      DeterministicAtomic,
+      std::chrono::steady_clock,
+      true>();
+}
+
+TEST(Baton, timed_wait_basic_steady_clock_multi_poster) {
+  run_basic_timed_wait_tests<std::atomic, std::chrono::steady_clock, false>();
+  run_basic_timed_wait_tests<
+      EmulatedFutexAtomic,
+      std::chrono::steady_clock,
+      false>();
+  run_basic_timed_wait_tests<
+      DeterministicAtomic,
+      std::chrono::steady_clock,
+      false>();
+}
+
+// Timed wait timeout steady clock tests
+
+TEST(Baton, timed_wait_timeout_steady_clock_single_poster) {
+  run_timed_wait_tmo_tests<std::atomic, std::chrono::steady_clock, true>();
+  run_timed_wait_tmo_tests<
+      EmulatedFutexAtomic,
+      std::chrono::steady_clock,
+      true>();
+  run_timed_wait_tmo_tests<
+      DeterministicAtomic,
+      std::chrono::steady_clock,
+      true>();
+}
+
+TEST(Baton, timed_wait_timeout_steady_clock_multi_poster) {
+  run_timed_wait_tmo_tests<std::atomic, std::chrono::steady_clock, false>();
+  run_timed_wait_tmo_tests<
+      EmulatedFutexAtomic,
+      std::chrono::steady_clock,
+      false>();
+  run_timed_wait_tmo_tests<
+      DeterministicAtomic,
+      std::chrono::steady_clock,
+      false>();
+}
+
+// Timed wait regular steady clock tests
+
+TEST(Baton, timed_wait_steady_clock_single_poster) {
+  run_timed_wait_regular_test<std::atomic, std::chrono::steady_clock, true>();
+  run_timed_wait_regular_test<
+      EmulatedFutexAtomic,
+      std::chrono::steady_clock,
+      true>();
+  run_timed_wait_regular_test<
+      DeterministicAtomic,
+      std::chrono::steady_clock,
+      true>();
+}
+
+TEST(Baton, timed_wait_steady_clock_multi_poster) {
+  run_timed_wait_regular_test<std::atomic, std::chrono::steady_clock, false>();
+  run_timed_wait_regular_test<
+      EmulatedFutexAtomic,
+      std::chrono::steady_clock,
+      false>();
+  run_timed_wait_regular_test<
+      DeterministicAtomic,
+      std::chrono::steady_clock,
+      false>();
+}
+
+/// Try wait tests
+
+TEST(Baton, try_wait_single_poster_blocking) {
+  run_try_wait_tests<std::atomic, true, true>();
+  run_try_wait_tests<EmulatedFutexAtomic, true, true>();
+  run_try_wait_tests<DeterministicAtomic, true, true>();
+}
+
+TEST(Baton, try_wait_single_poster_nonblocking) {
+  run_try_wait_tests<std::atomic, true, false>();
+  run_try_wait_tests<EmulatedFutexAtomic, true, false>();
+  run_try_wait_tests<DeterministicAtomic, true, false>();
+}
+
+TEST(Baton, try_wait_multi_poster_blocking) {
+  run_try_wait_tests<std::atomic, false, true>();
+  run_try_wait_tests<EmulatedFutexAtomic, false, true>();
+  run_try_wait_tests<DeterministicAtomic, false, true>();
+}
+
+TEST(Baton, try_wait_multi_poster_nonblocking) {
+  run_try_wait_tests<std::atomic, false, false>();
+  run_try_wait_tests<EmulatedFutexAtomic, false, false>();
+  run_try_wait_tests<DeterministicAtomic, false, false>();
+}
+
+/// Multi-producer tests
+
+TEST(Baton, multi_producer_single_poster_blocking) {
+  run_try_wait_tests<std::atomic, true, true>();
+  run_try_wait_tests<EmulatedFutexAtomic, true, true>();
+  run_try_wait_tests<DeterministicAtomic, true, true>();
+}
+
+TEST(Baton, multi_producer_single_poster_nonblocking) {
+  run_try_wait_tests<std::atomic, true, false>();
+  run_try_wait_tests<EmulatedFutexAtomic, true, false>();
+  run_try_wait_tests<DeterministicAtomic, true, false>();
+}
+
+TEST(Baton, multi_producer_multi_poster_blocking) {
+  run_try_wait_tests<std::atomic, false, true>();
+  run_try_wait_tests<EmulatedFutexAtomic, false, true>();
+  run_try_wait_tests<DeterministicAtomic, false, true>();
+}
+
+TEST(Baton, multi_producer_multi_poster_nonblocking) {
+  run_try_wait_tests<std::atomic, false, false>();
+  run_try_wait_tests<EmulatedFutexAtomic, false, false>();
+  run_try_wait_tests<DeterministicAtomic, false, false>();
+}
diff --git a/folly/synchronization/test/BatonTestHelpers.h b/folly/synchronization/test/BatonTestHelpers.h
new file mode 100644 (file)
index 0000000..6e242e2
--- /dev/null
@@ -0,0 +1,171 @@
+/*
+ * Copyright 2017 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 <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
+#include <folly/test/DeterministicSchedule.h>
+
+namespace folly {
+namespace test {
+
+typedef DeterministicSchedule DSched;
+
+template <template <typename> class Atom, bool SinglePoster, bool Blocking>
+void run_basic_test() {
+  Baton<Atom, SinglePoster, Blocking> b;
+  b.post();
+  b.wait();
+}
+
+template <template <typename> class Atom, bool SinglePoster, bool Blocking>
+void run_pingpong_test(int numRounds) {
+  using B = Baton<Atom, SinglePoster, Blocking>;
+  B batons[17];
+  B& a = batons[0];
+  B& b = batons[16]; // to get it on a different cache line
+  auto thr = DSched::thread([&] {
+    for (int i = 0; i < numRounds; ++i) {
+      a.wait();
+      a.reset();
+      b.post();
+    }
+  });
+  for (int i = 0; i < numRounds; ++i) {
+    a.post();
+    b.wait();
+    b.reset();
+  }
+  DSched::join(thr);
+}
+
+template <template <typename> class Atom, typename Clock, bool SinglePoster>
+void run_basic_timed_wait_tests() {
+  Baton<Atom, SinglePoster> b;
+  b.post();
+  // tests if early delivery works fine
+  EXPECT_TRUE(b.timed_wait(Clock::now()));
+}
+
+template <template <typename> class Atom, typename Clock, bool SinglePoster>
+void run_timed_wait_tmo_tests() {
+  Baton<Atom, SinglePoster> b;
+
+  auto thr = DSched::thread([&] {
+    bool rv = b.timed_wait(Clock::now() + std::chrono::milliseconds(1));
+    // main thread is guaranteed to not post until timeout occurs
+    EXPECT_FALSE(rv);
+  });
+  DSched::join(thr);
+}
+
+template <template <typename> class Atom, typename Clock, bool SinglePoster>
+void run_timed_wait_regular_test() {
+  Baton<Atom, SinglePoster> b;
+
+  auto thr = DSched::thread([&] {
+    // To wait forever we'd like to use time_point<Clock>::max, but
+    // std::condition_variable does math to convert the timeout to
+    // system_clock without handling overflow.
+    auto farFuture = Clock::now() + std::chrono::hours(1000);
+    bool rv = b.timed_wait(farFuture);
+    if (!std::is_same<Atom<int>, DeterministicAtomic<int>>::value) {
+      // DeterministicAtomic ignores actual times, so doesn't guarantee
+      // a lack of timeout
+      EXPECT_TRUE(rv);
+    }
+  });
+
+  if (!std::is_same<Atom<int>, DeterministicAtomic<int>>::value) {
+    // If we are using std::atomic (or EmulatedFutexAtomic) then
+    // a sleep here guarantees to a large extent that 'thr' will
+    // execute wait before we post it, thus testing late delivery. For
+    // DeterministicAtomic, we just rely on DeterministicSchedule to do
+    // the scheduling.  The test won't fail if we lose the race, we just
+    // don't get coverage.
+    std::this_thread::sleep_for(std::chrono::milliseconds(2));
+  }
+
+  b.post();
+  DSched::join(thr);
+}
+
+template <template <typename> class Atom, bool SinglePoster, bool Blocking>
+void run_try_wait_tests() {
+  Baton<Atom, SinglePoster, Blocking> b;
+  EXPECT_FALSE(b.try_wait());
+  b.post();
+  EXPECT_TRUE(b.try_wait());
+}
+
+template <template <typename> class Atom, bool SinglePoster, bool Blocking>
+void run_multi_producer_tests() {
+  constexpr int NPROD = 5;
+  Baton<Atom, SinglePoster, Blocking> local_ping[NPROD];
+  Baton<Atom, SinglePoster, Blocking> local_pong[NPROD];
+  Baton<Atom, /* SingleProducer = */ false, Blocking> global;
+  Baton<Atom, SinglePoster, Blocking> shutdown;
+
+  std::thread prod[NPROD];
+  for (int i = 0; i < NPROD; ++i) {
+    prod[i] = DSched::thread([&, i] {
+      if (!std::is_same<Atom<int>, DeterministicAtomic<int>>::value) {
+        // If we are using std::atomic (or EmulatedFutexAtomic) then
+        // a variable sleep here will make it more likely that
+        // global.post()-s will span more than one global.wait() by
+        // the consumer thread and for the latter to block (if the
+        // global baton is blocking). For DeterministicAtomic, we just
+        // rely on DeterministicSchedule to do the scheduling.  The
+        // test won't fail if we lose the race, we just don't get
+        // coverage.
+        for (int j = 0; j < i; ++j) {
+          std::this_thread::sleep_for(std::chrono::microseconds(1));
+        }
+      }
+      local_ping[i].post();
+      global.post();
+      local_pong[i].wait();
+    });
+  }
+
+  auto cons = DSched::thread([&] {
+    while (true) {
+      global.wait();
+      global.reset();
+      if (shutdown.try_wait()) {
+        return;
+      }
+      for (int i = 0; i < NPROD; ++i) {
+        if (local_ping.try_wait()) {
+          local_ping.reset();
+          local_pong.post();
+        }
+      }
+    }
+  });
+
+  for (auto& t : prod) {
+    DSched::join(t);
+  }
+
+  global.post();
+  shutdown.post();
+  DSched::join(cons);
+}
+
+} // namespace test
+} // namespace folly
index 9dce6bd9765f51e9ff92af0c70615ed9aa97b8cd..c4150247b5fccb07e5041d7032eb563cb0c7c637 100644 (file)
@@ -16,9 +16,9 @@
 
 #include <thread>
 
-#include <folly/Baton.h>
 #include <folly/ScopeGuard.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 #include <folly/system/ThreadName.h>
 
 using namespace std;
diff --git a/folly/test/BatonBenchmark.cpp b/folly/test/BatonBenchmark.cpp
deleted file mode 100644 (file)
index 82e3b5e..0000000
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Copyright 2017 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/Baton.h>
-
-#include <thread>
-
-#include <folly/Benchmark.h>
-#include <folly/portability/GFlags.h>
-#include <folly/portability/GTest.h>
-#include <folly/portability/Semaphore.h>
-#include <folly/test/BatonTestHelpers.h>
-#include <folly/test/DeterministicSchedule.h>
-
-using namespace folly;
-using namespace folly::test;
-using folly::detail::EmulatedFutexAtomic;
-
-typedef DeterministicSchedule DSched;
-
-BENCHMARK(baton_pingpong_single_poster_blocking, iters) {
-  run_pingpong_test<std::atomic, true, true>(iters);
-}
-
-BENCHMARK(baton_pingpong_multi_poster_blocking, iters) {
-  run_pingpong_test<std::atomic, false, true>(iters);
-}
-
-BENCHMARK(baton_pingpong_single_poster_nonblocking, iters) {
-  run_pingpong_test<std::atomic, true, false>(iters);
-}
-
-BENCHMARK(baton_pingpong_multi_poster_nonblocking, iters) {
-  run_pingpong_test<std::atomic, false, false>(iters);
-}
-
-BENCHMARK_DRAW_LINE()
-
-BENCHMARK(baton_pingpong_emulated_futex_single_poster_blocking, iters) {
-  run_pingpong_test<EmulatedFutexAtomic, true, true>(iters);
-}
-
-BENCHMARK(baton_pingpong_emulated_futex_multi_poster_blocking, iters) {
-  run_pingpong_test<EmulatedFutexAtomic, false, true>(iters);
-}
-
-BENCHMARK(baton_pingpong_emulated_futex_single_poster_nonblocking, iters) {
-  run_pingpong_test<EmulatedFutexAtomic, true, false>(iters);
-}
-
-BENCHMARK(baton_pingpong_emulated_futex_multi_poster_nonblocking, iters) {
-  run_pingpong_test<EmulatedFutexAtomic, false, false>(iters);
-}
-
-BENCHMARK_DRAW_LINE()
-
-BENCHMARK(posix_sem_pingpong, iters) {
-  sem_t sems[3];
-  sem_t* a = sems + 0;
-  sem_t* b = sems + 2; // to get it on a different cache line
-
-  sem_init(a, 0, 0);
-  sem_init(b, 0, 0);
-  auto thr = std::thread([=] {
-    for (size_t i = 0; i < iters; ++i) {
-      sem_wait(a);
-      sem_post(b);
-    }
-  });
-  for (size_t i = 0; i < iters; ++i) {
-    sem_post(a);
-    sem_wait(b);
-  }
-  thr.join();
-}
-
-// I am omitting a benchmark result snapshot because these microbenchmarks
-// mainly illustrate that PreBlockAttempts is very effective for rapid
-// handoffs.  The performance of Baton and sem_t is essentially identical
-// to the required futex calls for the blocking case
-
-int main(int argc, char** argv) {
-  testing::InitGoogleTest(&argc, argv);
-  gflags::ParseCommandLineFlags(&argc, &argv, true);
-
-  auto rv = RUN_ALL_TESTS();
-  if (!rv && FLAGS_benchmark) {
-    folly::runBenchmarks();
-  }
-  return rv;
-}
diff --git a/folly/test/BatonTest.cpp b/folly/test/BatonTest.cpp
deleted file mode 100644 (file)
index ded7fb5..0000000
+++ /dev/null
@@ -1,285 +0,0 @@
-/*
- * Copyright 2017 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/Baton.h>
-
-#include <thread>
-
-#include <folly/portability/GTest.h>
-#include <folly/test/BatonTestHelpers.h>
-#include <folly/test/DeterministicSchedule.h>
-
-using namespace folly;
-using namespace folly::test;
-using folly::detail::EmulatedFutexAtomic;
-
-/// Basic test
-
-TEST(Baton, basic_single_poster_blocking) {
-  run_basic_test<std::atomic, true, true>();
-  run_basic_test<EmulatedFutexAtomic, true, true>();
-  run_basic_test<DeterministicAtomic, true, true>();
-}
-
-TEST(Baton, basic_single_poster_nonblocking) {
-  run_basic_test<std::atomic, true, false>();
-  run_basic_test<EmulatedFutexAtomic, true, false>();
-  run_basic_test<DeterministicAtomic, true, false>();
-}
-
-TEST(Baton, basic_multi_poster_blocking) {
-  run_basic_test<std::atomic, false, true>();
-}
-
-TEST(Baton, basic_multi_poster_nonblocking) {
-  run_basic_test<std::atomic, false, false>();
-}
-
-/// Ping pong tests
-
-TEST(Baton, pingpong_single_poster_blocking) {
-  DSched sched(DSched::uniform(0));
-
-  run_pingpong_test<DeterministicAtomic, true, true>(1000);
-}
-
-TEST(Baton, pingpong_single_poster_nonblocking) {
-  DSched sched(DSched::uniform(0));
-
-  run_pingpong_test<DeterministicAtomic, true, false>(1000);
-}
-
-TEST(Baton, pingpong_multi_poster_blocking) {
-  DSched sched(DSched::uniform(0));
-
-  run_pingpong_test<DeterministicAtomic, false, true>(1000);
-}
-
-TEST(Baton, pingpong_multi_poster_nonblocking) {
-  DSched sched(DSched::uniform(0));
-
-  run_pingpong_test<DeterministicAtomic, false, false>(1000);
-}
-
-/// Timed wait tests - Nonblocking Baton does not support timed_wait()
-
-// Timed wait basic system clock tests
-
-TEST(Baton, timed_wait_basic_system_clock_single_poster) {
-  run_basic_timed_wait_tests<std::atomic, std::chrono::system_clock, true>();
-  run_basic_timed_wait_tests<
-      EmulatedFutexAtomic,
-      std::chrono::system_clock,
-      true>();
-  run_basic_timed_wait_tests<
-      DeterministicAtomic,
-      std::chrono::system_clock,
-      true>();
-}
-
-TEST(Baton, timed_wait_basic_system_clock_multi_poster) {
-  run_basic_timed_wait_tests<std::atomic, std::chrono::system_clock, false>();
-  run_basic_timed_wait_tests<
-      EmulatedFutexAtomic,
-      std::chrono::system_clock,
-      false>();
-  run_basic_timed_wait_tests<
-      DeterministicAtomic,
-      std::chrono::system_clock,
-      false>();
-}
-
-// Timed wait timeout system clock tests
-
-TEST(Baton, timed_wait_timeout_system_clock_single_poster) {
-  run_timed_wait_tmo_tests<std::atomic, std::chrono::system_clock, true>();
-  run_timed_wait_tmo_tests<
-      EmulatedFutexAtomic,
-      std::chrono::system_clock,
-      true>();
-  run_timed_wait_tmo_tests<
-      DeterministicAtomic,
-      std::chrono::system_clock,
-      true>();
-}
-
-TEST(Baton, timed_wait_timeout_system_clock_multi_poster) {
-  run_timed_wait_tmo_tests<std::atomic, std::chrono::system_clock, false>();
-  run_timed_wait_tmo_tests<
-      EmulatedFutexAtomic,
-      std::chrono::system_clock,
-      false>();
-  run_timed_wait_tmo_tests<
-      DeterministicAtomic,
-      std::chrono::system_clock,
-      false>();
-}
-
-// Timed wait regular system clock tests
-
-TEST(Baton, timed_wait_system_clock_single_poster) {
-  run_timed_wait_regular_test<std::atomic, std::chrono::system_clock, true>();
-  run_timed_wait_regular_test<
-      EmulatedFutexAtomic,
-      std::chrono::system_clock,
-      true>();
-  run_timed_wait_regular_test<
-      DeterministicAtomic,
-      std::chrono::system_clock,
-      true>();
-}
-
-TEST(Baton, timed_wait_system_clock_multi_poster) {
-  run_timed_wait_regular_test<std::atomic, std::chrono::system_clock, false>();
-  run_timed_wait_regular_test<
-      EmulatedFutexAtomic,
-      std::chrono::system_clock,
-      false>();
-  run_timed_wait_regular_test<
-      DeterministicAtomic,
-      std::chrono::system_clock,
-      false>();
-}
-
-// Timed wait basic steady clock tests
-
-TEST(Baton, timed_wait_basic_steady_clock_single_poster) {
-  run_basic_timed_wait_tests<std::atomic, std::chrono::steady_clock, true>();
-  run_basic_timed_wait_tests<
-      EmulatedFutexAtomic,
-      std::chrono::steady_clock,
-      true>();
-  run_basic_timed_wait_tests<
-      DeterministicAtomic,
-      std::chrono::steady_clock,
-      true>();
-}
-
-TEST(Baton, timed_wait_basic_steady_clock_multi_poster) {
-  run_basic_timed_wait_tests<std::atomic, std::chrono::steady_clock, false>();
-  run_basic_timed_wait_tests<
-      EmulatedFutexAtomic,
-      std::chrono::steady_clock,
-      false>();
-  run_basic_timed_wait_tests<
-      DeterministicAtomic,
-      std::chrono::steady_clock,
-      false>();
-}
-
-// Timed wait timeout steady clock tests
-
-TEST(Baton, timed_wait_timeout_steady_clock_single_poster) {
-  run_timed_wait_tmo_tests<std::atomic, std::chrono::steady_clock, true>();
-  run_timed_wait_tmo_tests<
-      EmulatedFutexAtomic,
-      std::chrono::steady_clock,
-      true>();
-  run_timed_wait_tmo_tests<
-      DeterministicAtomic,
-      std::chrono::steady_clock,
-      true>();
-}
-
-TEST(Baton, timed_wait_timeout_steady_clock_multi_poster) {
-  run_timed_wait_tmo_tests<std::atomic, std::chrono::steady_clock, false>();
-  run_timed_wait_tmo_tests<
-      EmulatedFutexAtomic,
-      std::chrono::steady_clock,
-      false>();
-  run_timed_wait_tmo_tests<
-      DeterministicAtomic,
-      std::chrono::steady_clock,
-      false>();
-}
-
-// Timed wait regular steady clock tests
-
-TEST(Baton, timed_wait_steady_clock_single_poster) {
-  run_timed_wait_regular_test<std::atomic, std::chrono::steady_clock, true>();
-  run_timed_wait_regular_test<
-      EmulatedFutexAtomic,
-      std::chrono::steady_clock,
-      true>();
-  run_timed_wait_regular_test<
-      DeterministicAtomic,
-      std::chrono::steady_clock,
-      true>();
-}
-
-TEST(Baton, timed_wait_steady_clock_multi_poster) {
-  run_timed_wait_regular_test<std::atomic, std::chrono::steady_clock, false>();
-  run_timed_wait_regular_test<
-      EmulatedFutexAtomic,
-      std::chrono::steady_clock,
-      false>();
-  run_timed_wait_regular_test<
-      DeterministicAtomic,
-      std::chrono::steady_clock,
-      false>();
-}
-
-/// Try wait tests
-
-TEST(Baton, try_wait_single_poster_blocking) {
-  run_try_wait_tests<std::atomic, true, true>();
-  run_try_wait_tests<EmulatedFutexAtomic, true, true>();
-  run_try_wait_tests<DeterministicAtomic, true, true>();
-}
-
-TEST(Baton, try_wait_single_poster_nonblocking) {
-  run_try_wait_tests<std::atomic, true, false>();
-  run_try_wait_tests<EmulatedFutexAtomic, true, false>();
-  run_try_wait_tests<DeterministicAtomic, true, false>();
-}
-
-TEST(Baton, try_wait_multi_poster_blocking) {
-  run_try_wait_tests<std::atomic, false, true>();
-  run_try_wait_tests<EmulatedFutexAtomic, false, true>();
-  run_try_wait_tests<DeterministicAtomic, false, true>();
-}
-
-TEST(Baton, try_wait_multi_poster_nonblocking) {
-  run_try_wait_tests<std::atomic, false, false>();
-  run_try_wait_tests<EmulatedFutexAtomic, false, false>();
-  run_try_wait_tests<DeterministicAtomic, false, false>();
-}
-
-/// Multi-producer tests
-
-TEST(Baton, multi_producer_single_poster_blocking) {
-  run_try_wait_tests<std::atomic, true, true>();
-  run_try_wait_tests<EmulatedFutexAtomic, true, true>();
-  run_try_wait_tests<DeterministicAtomic, true, true>();
-}
-
-TEST(Baton, multi_producer_single_poster_nonblocking) {
-  run_try_wait_tests<std::atomic, true, false>();
-  run_try_wait_tests<EmulatedFutexAtomic, true, false>();
-  run_try_wait_tests<DeterministicAtomic, true, false>();
-}
-
-TEST(Baton, multi_producer_multi_poster_blocking) {
-  run_try_wait_tests<std::atomic, false, true>();
-  run_try_wait_tests<EmulatedFutexAtomic, false, true>();
-  run_try_wait_tests<DeterministicAtomic, false, true>();
-}
-
-TEST(Baton, multi_producer_multi_poster_nonblocking) {
-  run_try_wait_tests<std::atomic, false, false>();
-  run_try_wait_tests<EmulatedFutexAtomic, false, false>();
-  run_try_wait_tests<DeterministicAtomic, false, false>();
-}
diff --git a/folly/test/BatonTestHelpers.h b/folly/test/BatonTestHelpers.h
deleted file mode 100644 (file)
index 3a48a21..0000000
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * Copyright 2017 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 <folly/Baton.h>
-#include <folly/portability/GTest.h>
-#include <folly/test/DeterministicSchedule.h>
-
-namespace folly {
-namespace test {
-
-typedef DeterministicSchedule DSched;
-
-template <template <typename> class Atom, bool SinglePoster, bool Blocking>
-void run_basic_test() {
-  Baton<Atom, SinglePoster, Blocking> b;
-  b.post();
-  b.wait();
-}
-
-template <template <typename> class Atom, bool SinglePoster, bool Blocking>
-void run_pingpong_test(int numRounds) {
-  using B = Baton<Atom, SinglePoster, Blocking>;
-  B batons[17];
-  B& a = batons[0];
-  B& b = batons[16]; // to get it on a different cache line
-  auto thr = DSched::thread([&] {
-    for (int i = 0; i < numRounds; ++i) {
-      a.wait();
-      a.reset();
-      b.post();
-    }
-  });
-  for (int i = 0; i < numRounds; ++i) {
-    a.post();
-    b.wait();
-    b.reset();
-  }
-  DSched::join(thr);
-}
-
-template <template <typename> class Atom, typename Clock, bool SinglePoster>
-void run_basic_timed_wait_tests() {
-  Baton<Atom, SinglePoster> b;
-  b.post();
-  // tests if early delivery works fine
-  EXPECT_TRUE(b.timed_wait(Clock::now()));
-}
-
-template <template <typename> class Atom, typename Clock, bool SinglePoster>
-void run_timed_wait_tmo_tests() {
-  Baton<Atom, SinglePoster> b;
-
-  auto thr = DSched::thread([&] {
-    bool rv = b.timed_wait(Clock::now() + std::chrono::milliseconds(1));
-    // main thread is guaranteed to not post until timeout occurs
-    EXPECT_FALSE(rv);
-  });
-  DSched::join(thr);
-}
-
-template <template <typename> class Atom, typename Clock, bool SinglePoster>
-void run_timed_wait_regular_test() {
-  Baton<Atom, SinglePoster> b;
-
-  auto thr = DSched::thread([&] {
-    // To wait forever we'd like to use time_point<Clock>::max, but
-    // std::condition_variable does math to convert the timeout to
-    // system_clock without handling overflow.
-    auto farFuture = Clock::now() + std::chrono::hours(1000);
-    bool rv = b.timed_wait(farFuture);
-    if (!std::is_same<Atom<int>, DeterministicAtomic<int>>::value) {
-      // DeterministicAtomic ignores actual times, so doesn't guarantee
-      // a lack of timeout
-      EXPECT_TRUE(rv);
-    }
-  });
-
-  if (!std::is_same<Atom<int>, DeterministicAtomic<int>>::value) {
-    // If we are using std::atomic (or EmulatedFutexAtomic) then
-    // a sleep here guarantees to a large extent that 'thr' will
-    // execute wait before we post it, thus testing late delivery. For
-    // DeterministicAtomic, we just rely on DeterministicSchedule to do
-    // the scheduling.  The test won't fail if we lose the race, we just
-    // don't get coverage.
-    std::this_thread::sleep_for(std::chrono::milliseconds(2));
-  }
-
-  b.post();
-  DSched::join(thr);
-}
-
-template <template <typename> class Atom, bool SinglePoster, bool Blocking>
-void run_try_wait_tests() {
-  Baton<Atom, SinglePoster, Blocking> b;
-  EXPECT_FALSE(b.try_wait());
-  b.post();
-  EXPECT_TRUE(b.try_wait());
-}
-
-template <template <typename> class Atom, bool SinglePoster, bool Blocking>
-void run_multi_producer_tests() {
-  constexpr int NPROD = 5;
-  Baton<Atom, SinglePoster, Blocking> local_ping[NPROD];
-  Baton<Atom, SinglePoster, Blocking> local_pong[NPROD];
-  Baton<Atom, /* SingleProducer = */ false, Blocking> global;
-  Baton<Atom, SinglePoster, Blocking> shutdown;
-
-  std::thread prod[NPROD];
-  for (int i = 0; i < NPROD; ++i) {
-    prod[i] = DSched::thread([&, i] {
-      if (!std::is_same<Atom<int>, DeterministicAtomic<int>>::value) {
-        // If we are using std::atomic (or EmulatedFutexAtomic) then
-        // a variable sleep here will make it more likely that
-        // global.post()-s will span more than one global.wait() by
-        // the consumer thread and for the latter to block (if the
-        // global baton is blocking). For DeterministicAtomic, we just
-        // rely on DeterministicSchedule to do the scheduling.  The
-        // test won't fail if we lose the race, we just don't get
-        // coverage.
-        for (int j = 0; j < i; ++j) {
-          std::this_thread::sleep_for(std::chrono::microseconds(1));
-        }
-      }
-      local_ping[i].post();
-      global.post();
-      local_pong[i].wait();
-    });
-  }
-
-  auto cons = DSched::thread([&] {
-    while (true) {
-      global.wait();
-      global.reset();
-      if (shutdown.try_wait()) {
-        return;
-      }
-      for (int i = 0; i < NPROD; ++i) {
-        if (local_ping.try_wait()) {
-          local_ping.reset();
-          local_pong.post();
-        }
-      }
-    }
-  });
-
-  for (auto& t : prod) {
-    DSched::join(t);
-  }
-
-  global.post();
-  shutdown.post();
-  DSched::join(cons);
-}
-
-} // namespace test
-} // namespace folly
index 72546f41e58ffab5aec97e063bab020da706b495..d5393115dad7995598712540499e34cf38ee6231 100644 (file)
@@ -16,9 +16,9 @@
 
 #include <folly/detail/MemoryIdler.h>
 
-#include <folly/Baton.h>
 #include <folly/portability/GMock.h>
 #include <folly/portability/GTest.h>
+#include <folly/synchronization/Baton.h>
 
 #include <memory>
 #include <thread>
index 6791c0f42902d1a4c731c9cfc3efb40a447eb4a6..e724b999604e105200707a342526311a1ac77cf2 100644 (file)
 
 #include <glog/logging.h>
 
-#include <folly/Baton.h>
 #include <folly/Memory.h>
 #include <folly/experimental/io/FsUtil.h>
 #include <folly/portability/GTest.h>
 #include <folly/portability/Unistd.h>
+#include <folly/synchronization/Baton.h>
 #include <folly/system/ThreadId.h>
 
 using namespace folly;