apply_folly_compile_options_to_target(folly_test_support)
folly_define_tests(
+ DIRECTORY concurrency/
+ TEST cache_locality_test SOURCES CacheLocalityTest.cpp
DIRECTORY experimental/test/
TEST autotimer_test SOURCES AutoTimerTest.cpp
TEST bits_test_2 SOURCES BitsTest.cpp
TEST baton_test SOURCES BatonTest.cpp
TEST bit_iterator_test SOURCES BitIteratorTest.cpp
TEST bits_test SOURCES BitsTest.cpp
- TEST cache_locality_test SOURCES CacheLocalityTest.cpp
TEST cacheline_padded_test SOURCES CachelinePaddedTest.cpp
TEST call_once_test SOURCES CallOnceTest.cpp
TEST checksum_test SOURCES ChecksumTest.cpp
#pragma once
-#include <type_traits>
#include <assert.h>
#include <errno.h>
#include <stdint.h>
+
+#include <type_traits>
+
#include <boost/noncopyable.hpp>
#include <folly/AtomicStruct.h>
#include <folly/Portability.h>
-#include <folly/detail/CacheLocality.h>
+#include <folly/concurrency/CacheLocality.h>
#include <folly/portability/SysMman.h>
#include <folly/portability/Unistd.h>
}
AtomicStruct<TaggedPtr,Atom>& localHead() {
- auto stripe = detail::AccessSpreader<Atom>::current(NumLocalLists);
+ auto stripe = AccessSpreader<Atom>::current(NumLocalLists);
return local_[stripe].head;
}
#include <folly/Baton.h>
#include <folly/IndexedMemPool.h>
#include <folly/Likely.h>
-#include <folly/detail/CacheLocality.h>
+#include <folly/concurrency/CacheLocality.h>
namespace folly {
FOLLY_ALIGN_TO_AVOID_FALSE_SHARING
folly::AtomicStruct<LifoSemHead,Atom> head_;
- char padding_[folly::detail::CacheLocality::kFalseSharingRange -
- sizeof(LifoSemHead)];
-
+ char padding_[folly::CacheLocality::kFalseSharingRange - sizeof(LifoSemHead)];
static LifoSemNode<Handoff, Atom>& idxToNode(uint32_t idx) {
auto raw = &LifoSemRawNode<Atom>::pool()[idx];
#include <type_traits>
#include <folly/Traits.h>
-#include <folly/detail/CacheLocality.h>
+#include <folly/concurrency/CacheLocality.h>
#include <folly/detail/TurnSequencer.h>
#include <folly/portability/Unistd.h>
}
// ideally this would be a static assert, but g++ doesn't allow it
- assert(alignof(MPMCQueue<T,Atom>)
- >= detail::CacheLocality::kFalseSharingRange);
- assert(static_cast<uint8_t*>(static_cast<void*>(&popTicket_))
- - static_cast<uint8_t*>(static_cast<void*>(&pushTicket_))
- >= detail::CacheLocality::kFalseSharingRange);
+ assert(alignof(MPMCQueue<T, Atom>) >= CacheLocality::kFalseSharingRange);
+ assert(
+ static_cast<uint8_t*>(static_cast<void*>(&popTicket_)) -
+ static_cast<uint8_t*>(static_cast<void*>(&pushTicket_)) >=
+ CacheLocality::kFalseSharingRange);
}
/// A default-constructed queue is useful because a usable (non-zero
/// To avoid false sharing in slots_ with neighboring memory
/// allocations, we pad it with this many SingleElementQueue-s at
/// each end
- kSlotPadding = (detail::CacheLocality::kFalseSharingRange - 1)
- / sizeof(Slot) + 1
+ kSlotPadding = (CacheLocality::kFalseSharingRange - 1) / sizeof(Slot) + 1
};
/// The maximum number of items in the queue at once
/// Alignment doesn't prevent false sharing at the end of the struct,
/// so fill out the last cache line
- char padding_[detail::CacheLocality::kFalseSharingRange -
- sizeof(Atom<uint32_t>)];
+ char padding_[CacheLocality::kFalseSharingRange - sizeof(Atom<uint32_t>)];
/// We assign tickets in increasing order, but we don't want to
/// access neighboring elements of slots_ because that will lead to
CppAttributes.h \
CpuId.h \
CPortability.h \
+ concurrency/CacheLocality.h \
concurrency/CoreCachedSharedPtr.h \
detail/AtomicHashUtils.h \
detail/AtomicUnorderedMapUtils.h \
detail/AtomicUtils.h \
detail/BitIteratorDetail.h \
- detail/CacheLocality.h \
detail/CachelinePaddedImpl.h \
detail/ChecksumDetail.h \
detail/DiscriminatedPtrDetail.h \
Assume.cpp \
Checksum.cpp \
ClockGettimeWrappers.cpp \
- detail/CacheLocality.cpp \
+ concurrency/CacheLocality.cpp \
detail/IPAddress.cpp \
dynamic.cpp \
ExceptionWrapper.cpp \
#include <type_traits>
#include <utility>
-#include <folly/detail/CacheLocality.h>
+#include <folly/concurrency/CacheLocality.h>
namespace folly {
}
private:
- char pad0_[detail::CacheLocality::kFalseSharingRange];
- const uint32_t size_;
- T* const records_;
+ char pad0_[CacheLocality::kFalseSharingRange];
+ const uint32_t size_;
+ T* const records_;
- FOLLY_ALIGN_TO_AVOID_FALSE_SHARING std::atomic<unsigned int> readIndex_;
- FOLLY_ALIGN_TO_AVOID_FALSE_SHARING std::atomic<unsigned int> writeIndex_;
+ FOLLY_ALIGN_TO_AVOID_FALSE_SHARING std::atomic<unsigned int> readIndex_;
+ FOLLY_ALIGN_TO_AVOID_FALSE_SHARING std::atomic<unsigned int> writeIndex_;
- char pad1_[detail::CacheLocality::kFalseSharingRange - sizeof(writeIndex_)];
+ char pad1_[CacheLocality::kFalseSharingRange - sizeof(writeIndex_)];
};
}
#pragma once
#include <stdint.h>
+
#include <atomic>
#include <thread>
#include <type_traits>
+
#include <folly/Likely.h>
-#include <folly/detail/CacheLocality.h>
+#include <folly/concurrency/CacheLocality.h>
#include <folly/detail/Futex.h>
#include <folly/portability/Asm.h>
#include <folly/portability/SysResource.h>
// starting point for our empty-slot search, can change after
// calling waitForZeroBits
uint32_t bestSlot =
- (uint32_t)folly::detail::AccessSpreader<Atom>::current(
- kMaxDeferredReaders);
+ (uint32_t)folly::AccessSpreader<Atom>::current(kMaxDeferredReaders);
// deferred readers are already enabled, or it is time to
// enable them if we can find a slot
#include <chrono>
#include <folly/Likely.h>
-#include <folly/detail/CacheLocality.h>
+#include <folly/concurrency/CacheLocality.h>
namespace folly {
--- /dev/null
+/*
+ * 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/concurrency/CacheLocality.h>
+
+#ifndef _MSC_VER
+#define _GNU_SOURCE 1 // for RTLD_NOLOAD
+#include <dlfcn.h>
+#endif
+#include <fstream>
+
+#include <folly/Conv.h>
+#include <folly/Exception.h>
+#include <folly/FileUtil.h>
+#include <folly/Format.h>
+#include <folly/ScopeGuard.h>
+
+namespace folly {
+
+///////////// CacheLocality
+
+/// Returns the best real CacheLocality information available
+static CacheLocality getSystemLocalityInfo() {
+#ifdef __linux__
+ try {
+ return CacheLocality::readFromSysfs();
+ } catch (...) {
+ // keep trying
+ }
+#endif
+
+ long numCpus = sysconf(_SC_NPROCESSORS_CONF);
+ if (numCpus <= 0) {
+ // This shouldn't happen, but if it does we should try to keep
+ // going. We are probably not going to be able to parse /sys on
+ // this box either (although we will try), which means we are going
+ // to fall back to the SequentialThreadId splitter. On my 16 core
+ // (x hyperthreading) dev box 16 stripes is enough to get pretty good
+ // contention avoidance with SequentialThreadId, and there is little
+ // improvement from going from 32 to 64. This default gives us some
+ // wiggle room
+ numCpus = 32;
+ }
+ return CacheLocality::uniform(size_t(numCpus));
+}
+
+template <>
+const CacheLocality& CacheLocality::system<std::atomic>() {
+ static auto* cache = new CacheLocality(getSystemLocalityInfo());
+ return *cache;
+}
+
+// Each level of cache has sharing sets, which are the set of cpus
+// that share a common cache at that level. These are available in a
+// hex bitset form (/sys/devices/system/cpu/cpu0/index0/shared_cpu_map,
+// for example). They are also available in a human-readable list form,
+// as in /sys/devices/system/cpu/cpu0/index0/shared_cpu_list. The list
+// is a comma-separated list of numbers and ranges, where the ranges are
+// a pair of decimal numbers separated by a '-'.
+//
+// To sort the cpus for optimum locality we don't really need to parse
+// the sharing sets, we just need a unique representative from the
+// equivalence class. The smallest value works fine, and happens to be
+// the first decimal number in the file. We load all of the equivalence
+// class information from all of the cpu*/index* directories, order the
+// cpus first by increasing last-level cache equivalence class, then by
+// the smaller caches. Finally, we break ties with the cpu number itself.
+
+/// Returns the first decimal number in the string, or throws an exception
+/// if the string does not start with a number terminated by ',', '-',
+/// '\n', or eos.
+static size_t parseLeadingNumber(const std::string& line) {
+ auto raw = line.c_str();
+ char* end;
+ unsigned long val = strtoul(raw, &end, 10);
+ if (end == raw || (*end != ',' && *end != '-' && *end != '\n' && *end != 0)) {
+ throw std::runtime_error(
+ to<std::string>("error parsing list '", line, "'").c_str());
+ }
+ return val;
+}
+
+CacheLocality CacheLocality::readFromSysfsTree(
+ const std::function<std::string(std::string)>& mapping) {
+ // number of equivalence classes per level
+ std::vector<size_t> numCachesByLevel;
+
+ // the list of cache equivalence classes, where equivalance classes
+ // are named by the smallest cpu in the class
+ std::vector<std::vector<size_t>> equivClassesByCpu;
+
+ std::vector<size_t> cpus;
+
+ while (true) {
+ auto cpu = cpus.size();
+ std::vector<size_t> levels;
+ for (size_t index = 0;; ++index) {
+ auto dir =
+ sformat("/sys/devices/system/cpu/cpu{}/cache/index{}/", cpu, index);
+ auto cacheType = mapping(dir + "type");
+ auto equivStr = mapping(dir + "shared_cpu_list");
+ if (cacheType.size() == 0 || equivStr.size() == 0) {
+ // no more caches
+ break;
+ }
+ if (cacheType[0] == 'I') {
+ // cacheType in { "Data", "Instruction", "Unified" }. skip icache
+ continue;
+ }
+ auto equiv = parseLeadingNumber(equivStr);
+ auto level = levels.size();
+ levels.push_back(equiv);
+
+ if (equiv == cpu) {
+ // we only want to count the equiv classes once, so we do it when
+ // we first encounter them
+ while (numCachesByLevel.size() <= level) {
+ numCachesByLevel.push_back(0);
+ }
+ numCachesByLevel[level]++;
+ }
+ }
+
+ if (levels.size() == 0) {
+ // no levels at all for this cpu, we must be done
+ break;
+ }
+ equivClassesByCpu.emplace_back(std::move(levels));
+ cpus.push_back(cpu);
+ }
+
+ if (cpus.size() == 0) {
+ throw std::runtime_error("unable to load cache sharing info");
+ }
+
+ std::sort(cpus.begin(),
+ cpus.end(),
+ [&](size_t lhs, size_t rhs) -> bool {
+ // sort first by equiv class of cache with highest index,
+ // direction doesn't matter. If different cpus have
+ // different numbers of caches then this code might produce
+ // a sub-optimal ordering, but it won't crash
+ auto& lhsEquiv = equivClassesByCpu[lhs];
+ auto& rhsEquiv = equivClassesByCpu[rhs];
+ for (ssize_t i = ssize_t(std::min(lhsEquiv.size(), rhsEquiv.size())) - 1;
+ i >= 0;
+ --i) {
+ auto idx = size_t(i);
+ if (lhsEquiv[idx] != rhsEquiv[idx]) {
+ return lhsEquiv[idx] < rhsEquiv[idx];
+ }
+ }
+
+ // break ties deterministically by cpu
+ return lhs < rhs;
+ });
+
+ // the cpus are now sorted by locality, with neighboring entries closer
+ // to each other than entries that are far away. For striping we want
+ // the inverse map, since we are starting with the cpu
+ std::vector<size_t> indexes(cpus.size());
+ for (size_t i = 0; i < cpus.size(); ++i) {
+ indexes[cpus[i]] = i;
+ }
+
+ return CacheLocality{
+ cpus.size(), std::move(numCachesByLevel), std::move(indexes)};
+}
+
+CacheLocality CacheLocality::readFromSysfs() {
+ return readFromSysfsTree([](std::string name) {
+ std::ifstream xi(name.c_str());
+ std::string rv;
+ std::getline(xi, rv);
+ return rv;
+ });
+}
+
+CacheLocality CacheLocality::uniform(size_t numCpus) {
+ CacheLocality rv;
+
+ rv.numCpus = numCpus;
+
+ // one cache shared by all cpus
+ rv.numCachesByLevel.push_back(numCpus);
+
+ // no permutations in locality index mapping
+ for (size_t cpu = 0; cpu < numCpus; ++cpu) {
+ rv.localityIndexByCpu.push_back(cpu);
+ }
+
+ return rv;
+}
+
+////////////// Getcpu
+
+Getcpu::Func Getcpu::resolveVdsoFunc() {
+#if !FOLLY_HAVE_LINUX_VDSO
+ return nullptr;
+#else
+ void* h = dlopen("linux-vdso.so.1", RTLD_LAZY | RTLD_LOCAL | RTLD_NOLOAD);
+ if (h == nullptr) {
+ return nullptr;
+ }
+
+ auto func = Getcpu::Func(dlsym(h, "__vdso_getcpu"));
+ if (func == nullptr) {
+ // technically a null result could either be a failure or a successful
+ // lookup of a symbol with the null value, but the second can't actually
+ // happen for this symbol. No point holding the handle forever if
+ // we don't need the code
+ dlclose(h);
+ }
+
+ return func;
+#endif
+}
+
+#ifdef FOLLY_TLS
+/////////////// SequentialThreadId
+template struct SequentialThreadId<std::atomic>;
+#endif
+
+/////////////// AccessSpreader
+template struct AccessSpreader<std::atomic>;
+
+SimpleAllocator::SimpleAllocator(size_t allocSize, size_t sz)
+ : allocSize_{allocSize}, sz_(sz) {}
+
+SimpleAllocator::~SimpleAllocator() {
+ std::lock_guard<std::mutex> g(m_);
+ for (auto& block : blocks_) {
+ detail::aligned_free(block);
+ }
+}
+
+void* SimpleAllocator::allocateHard() {
+ // Allocate a new slab.
+ mem_ = static_cast<uint8_t*>(detail::aligned_malloc(allocSize_, allocSize_));
+ if (!mem_) {
+ std::__throw_bad_alloc();
+ }
+ end_ = mem_ + allocSize_;
+ blocks_.push_back(mem_);
+
+ // Install a pointer to ourselves as the allocator.
+ *reinterpret_cast<SimpleAllocator**>(mem_) = this;
+ static_assert(
+ alignof(std::max_align_t) >= sizeof(SimpleAllocator*),
+ "alignment too small");
+ mem_ += std::min(sz_, alignof(std::max_align_t));
+
+ // New allocation.
+ auto mem = mem_;
+ mem_ += sz_;
+ assert(intptr_t(mem) % 128 != 0);
+ return mem;
+}
+
+} // namespace folly
--- /dev/null
+/*
+ * 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 <algorithm>
+#include <array>
+#include <atomic>
+#include <cassert>
+#include <functional>
+#include <limits>
+#include <mutex>
+#include <string>
+#include <type_traits>
+#include <unordered_map>
+#include <vector>
+
+#include <folly/Hash.h>
+#include <folly/Indestructible.h>
+#include <folly/Likely.h>
+#include <folly/Memory.h>
+#include <folly/Portability.h>
+#include <folly/ThreadId.h>
+#include <folly/portability/BitsFunctexcept.h>
+#include <folly/portability/Memory.h>
+
+namespace folly {
+
+// This file contains several classes that might be useful if you are
+// trying to dynamically optimize cache locality: CacheLocality reads
+// cache sharing information from sysfs to determine how CPUs should be
+// grouped to minimize contention, Getcpu provides fast access to the
+// current CPU via __vdso_getcpu, and AccessSpreader uses these two to
+// optimally spread accesses among a predetermined number of stripes.
+//
+// AccessSpreader<>::current(n) microbenchmarks at 22 nanos, which is
+// substantially less than the cost of a cache miss. This means that we
+// can effectively use it to reduce cache line ping-pong on striped data
+// structures such as IndexedMemPool or statistics counters.
+//
+// Because CacheLocality looks at all of the cache levels, it can be
+// used for different levels of optimization. AccessSpreader(2) does
+// per-chip spreading on a dual socket system. AccessSpreader(numCpus)
+// does perfect per-cpu spreading. AccessSpreader(numCpus / 2) does
+// perfect L1 spreading in a system with hyperthreading enabled.
+
+struct CacheLocality {
+
+ /// 1 more than the maximum value that can be returned from sched_getcpu
+ /// or getcpu. This is the number of hardware thread contexts provided
+ /// by the processors
+ size_t numCpus;
+
+ /// Holds the number of caches present at each cache level (0 is
+ /// the closest to the cpu). This is the number of AccessSpreader
+ /// stripes needed to avoid cross-cache communication at the specified
+ /// layer. numCachesByLevel.front() is the number of L1 caches and
+ /// numCachesByLevel.back() is the number of last-level caches.
+ std::vector<size_t> numCachesByLevel;
+
+ /// A map from cpu (from sched_getcpu or getcpu) to an index in the
+ /// range 0..numCpus-1, where neighboring locality indices are more
+ /// likely to share caches then indices far away. All of the members
+ /// of a particular cache level be contiguous in their locality index.
+ /// For example, if numCpus is 32 and numCachesByLevel.back() is 2,
+ /// then cpus with a locality index < 16 will share one last-level
+ /// cache and cpus with a locality index >= 16 will share the other.
+ std::vector<size_t> localityIndexByCpu;
+
+ /// Returns the best CacheLocality information available for the current
+ /// system, cached for fast access. This will be loaded from sysfs if
+ /// possible, otherwise it will be correct in the number of CPUs but
+ /// not in their sharing structure.
+ ///
+ /// If you are into yo dawgs, this is a shared cache of the local
+ /// locality of the shared caches.
+ ///
+ /// The template parameter here is used to allow injection of a
+ /// repeatable CacheLocality structure during testing. Rather than
+ /// inject the type of the CacheLocality provider into every data type
+ /// that transitively uses it, all components select between the default
+ /// sysfs implementation and a deterministic implementation by keying
+ /// off the type of the underlying atomic. See DeterministicScheduler.
+ template <template <typename> class Atom = std::atomic>
+ static const CacheLocality& system();
+
+ /// Reads CacheLocality information from a tree structured like
+ /// the sysfs filesystem. The provided function will be evaluated
+ /// for each sysfs file that needs to be queried. The function
+ /// should return a string containing the first line of the file
+ /// (not including the newline), or an empty string if the file does
+ /// not exist. The function will be called with paths of the form
+ /// /sys/devices/system/cpu/cpu*/cache/index*/{type,shared_cpu_list} .
+ /// Throws an exception if no caches can be parsed at all.
+ static CacheLocality readFromSysfsTree(
+ const std::function<std::string(std::string)>& mapping);
+
+ /// Reads CacheLocality information from the real sysfs filesystem.
+ /// Throws an exception if no cache information can be loaded.
+ static CacheLocality readFromSysfs();
+
+ /// Returns a usable (but probably not reflective of reality)
+ /// CacheLocality structure with the specified number of cpus and a
+ /// single cache level that associates one cpu per cache.
+ static CacheLocality uniform(size_t numCpus);
+
+ enum {
+ /// Memory locations on the same cache line are subject to false
+ /// sharing, which is very bad for performance. Microbenchmarks
+ /// indicate that pairs of cache lines also see interference under
+ /// heavy use of atomic operations (observed for atomic increment on
+ /// Sandy Bridge). See FOLLY_ALIGN_TO_AVOID_FALSE_SHARING
+ kFalseSharingRange = 128
+ };
+
+ static_assert(
+ kFalseSharingRange == 128,
+ "FOLLY_ALIGN_TO_AVOID_FALSE_SHARING should track kFalseSharingRange");
+};
+
+// TODO replace __attribute__ with alignas and 128 with kFalseSharingRange
+
+/// An attribute that will cause a variable or field to be aligned so that
+/// it doesn't have false sharing with anything at a smaller memory address.
+#define FOLLY_ALIGN_TO_AVOID_FALSE_SHARING FOLLY_ALIGNED(128)
+
+/// Knows how to derive a function pointer to the VDSO implementation of
+/// getcpu(2), if available
+struct Getcpu {
+ /// Function pointer to a function with the same signature as getcpu(2).
+ typedef int (*Func)(unsigned* cpu, unsigned* node, void* unused);
+
+ /// Returns a pointer to the VDSO implementation of getcpu(2), if
+ /// available, or nullptr otherwise. This function may be quite
+ /// expensive, be sure to cache the result.
+ static Func resolveVdsoFunc();
+};
+
+#ifdef FOLLY_TLS
+template <template <typename> class Atom>
+struct SequentialThreadId {
+
+ /// Returns the thread id assigned to the current thread
+ static unsigned get() {
+ auto rv = currentId;
+ if (UNLIKELY(rv == 0)) {
+ rv = currentId = ++prevId;
+ }
+ return rv;
+ }
+
+ private:
+ static Atom<unsigned> prevId;
+
+ static FOLLY_TLS unsigned currentId;
+};
+
+template <template <typename> class Atom>
+Atom<unsigned> SequentialThreadId<Atom>::prevId(0);
+
+template <template <typename> class Atom>
+FOLLY_TLS unsigned SequentialThreadId<Atom>::currentId(0);
+
+// Suppress this instantiation in other translation units. It is
+// instantiated in CacheLocality.cpp
+extern template struct SequentialThreadId<std::atomic>;
+#endif
+
+struct HashingThreadId {
+ static unsigned get() {
+ return hash::twang_32from64(getCurrentThreadID());
+ }
+};
+
+/// A class that lazily binds a unique (for each implementation of Atom)
+/// identifier to a thread. This is a fallback mechanism for the access
+/// spreader if __vdso_getcpu can't be loaded
+template <typename ThreadId>
+struct FallbackGetcpu {
+ /// Fills the thread id into the cpu and node out params (if they
+ /// are non-null). This method is intended to act like getcpu when a
+ /// fast-enough form of getcpu isn't available or isn't desired
+ static int getcpu(unsigned* cpu, unsigned* node, void* /* unused */) {
+ auto id = ThreadId::get();
+ if (cpu) {
+ *cpu = id;
+ }
+ if (node) {
+ *node = id;
+ }
+ return 0;
+ }
+};
+
+#ifdef FOLLY_TLS
+typedef FallbackGetcpu<SequentialThreadId<std::atomic>> FallbackGetcpuType;
+#else
+typedef FallbackGetcpu<HashingThreadId> FallbackGetcpuType;
+#endif
+
+/// AccessSpreader arranges access to a striped data structure in such a
+/// way that concurrently executing threads are likely to be accessing
+/// different stripes. It does NOT guarantee uncontended access.
+/// Your underlying algorithm must be thread-safe without spreading, this
+/// is merely an optimization. AccessSpreader::current(n) is typically
+/// much faster than a cache miss (12 nanos on my dev box, tested fast
+/// in both 2.6 and 3.2 kernels).
+///
+/// If available (and not using the deterministic testing implementation)
+/// AccessSpreader uses the getcpu system call via VDSO and the
+/// precise locality information retrieved from sysfs by CacheLocality.
+/// This provides optimal anti-sharing at a fraction of the cost of a
+/// cache miss.
+///
+/// When there are not as many stripes as processors, we try to optimally
+/// place the cache sharing boundaries. This means that if you have 2
+/// stripes and run on a dual-socket system, your 2 stripes will each get
+/// all of the cores from a single socket. If you have 16 stripes on a
+/// 16 core system plus hyperthreading (32 cpus), each core will get its
+/// own stripe and there will be no cache sharing at all.
+///
+/// AccessSpreader has a fallback mechanism for when __vdso_getcpu can't be
+/// loaded, or for use during deterministic testing. Using sched_getcpu
+/// or the getcpu syscall would negate the performance advantages of
+/// access spreading, so we use a thread-local value and a shared atomic
+/// counter to spread access out. On systems lacking both a fast getcpu()
+/// and TLS, we hash the thread id to spread accesses.
+///
+/// AccessSpreader is templated on the template type that is used
+/// to implement atomics, as a way to instantiate the underlying
+/// heuristics differently for production use and deterministic unit
+/// testing. See DeterministicScheduler for more. If you aren't using
+/// DeterministicScheduler, you can just use the default template parameter
+/// all of the time.
+template <template <typename> class Atom = std::atomic>
+struct AccessSpreader {
+
+ /// Returns the stripe associated with the current CPU. The returned
+ /// value will be < numStripes.
+ static size_t current(size_t numStripes) {
+ // widthAndCpuToStripe[0] will actually work okay (all zeros), but
+ // something's wrong with the caller
+ assert(numStripes > 0);
+
+ unsigned cpu;
+ getcpuFunc(&cpu, nullptr, nullptr);
+ return widthAndCpuToStripe[std::min(size_t(kMaxCpus),
+ numStripes)][cpu % kMaxCpus];
+ }
+
+ private:
+ /// If there are more cpus than this nothing will crash, but there
+ /// might be unnecessary sharing
+ enum { kMaxCpus = 128 };
+
+ typedef uint8_t CompactStripe;
+
+ static_assert((kMaxCpus & (kMaxCpus - 1)) == 0,
+ "kMaxCpus should be a power of two so modulo is fast");
+ static_assert(kMaxCpus - 1 <= std::numeric_limits<CompactStripe>::max(),
+ "stripeByCpu element type isn't wide enough");
+
+ /// Points to the getcpu-like function we are using to obtain the
+ /// current cpu. It should not be assumed that the returned cpu value
+ /// is in range. We use a static for this so that we can prearrange a
+ /// valid value in the pre-constructed state and avoid the need for a
+ /// conditional on every subsequent invocation (not normally a big win,
+ /// but 20% on some inner loops here).
+ static Getcpu::Func getcpuFunc;
+
+ /// For each level of splitting up to kMaxCpus, maps the cpu (mod
+ /// kMaxCpus) to the stripe. Rather than performing any inequalities
+ /// or modulo on the actual number of cpus, we just fill in the entire
+ /// array.
+ static CompactStripe widthAndCpuToStripe[kMaxCpus + 1][kMaxCpus];
+
+ static bool initialized;
+
+ /// Returns the best getcpu implementation for Atom
+ static Getcpu::Func pickGetcpuFunc() {
+ auto best = Getcpu::resolveVdsoFunc();
+ return best ? best : &FallbackGetcpuType::getcpu;
+ }
+
+ /// Always claims to be on CPU zero, node zero
+ static int degenerateGetcpu(unsigned* cpu, unsigned* node, void*) {
+ if (cpu != nullptr) {
+ *cpu = 0;
+ }
+ if (node != nullptr) {
+ *node = 0;
+ }
+ return 0;
+ }
+
+ // The function to call for fast lookup of getcpu is a singleton, as
+ // is the precomputed table of locality information. AccessSpreader
+ // is used in very tight loops, however (we're trying to race an L1
+ // cache miss!), so the normal singleton mechanisms are noticeably
+ // expensive. Even a not-taken branch guarding access to getcpuFunc
+ // slows AccessSpreader::current from 12 nanos to 14. As a result, we
+ // populate the static members with simple (but valid) values that can
+ // be filled in by the linker, and then follow up with a normal static
+ // initializer call that puts in the proper version. This means that
+ // when there are initialization order issues we will just observe a
+ // zero stripe. Once a sanitizer gets smart enough to detect this as
+ // a race or undefined behavior, we can annotate it.
+
+ static bool initialize() {
+ getcpuFunc = pickGetcpuFunc();
+
+ auto& cacheLocality = CacheLocality::system<Atom>();
+ auto n = cacheLocality.numCpus;
+ for (size_t width = 0; width <= kMaxCpus; ++width) {
+ auto numStripes = std::max(size_t{1}, width);
+ for (size_t cpu = 0; cpu < kMaxCpus && cpu < n; ++cpu) {
+ auto index = cacheLocality.localityIndexByCpu[cpu];
+ assert(index < n);
+ // as index goes from 0..n, post-transform value goes from
+ // 0..numStripes
+ widthAndCpuToStripe[width][cpu] =
+ CompactStripe((index * numStripes) / n);
+ assert(widthAndCpuToStripe[width][cpu] < numStripes);
+ }
+ for (size_t cpu = n; cpu < kMaxCpus; ++cpu) {
+ widthAndCpuToStripe[width][cpu] = widthAndCpuToStripe[width][cpu - n];
+ }
+ }
+ return true;
+ }
+};
+
+template <template <typename> class Atom>
+Getcpu::Func AccessSpreader<Atom>::getcpuFunc =
+ AccessSpreader<Atom>::degenerateGetcpu;
+
+template <template <typename> class Atom>
+typename AccessSpreader<Atom>::CompactStripe
+ AccessSpreader<Atom>::widthAndCpuToStripe[kMaxCpus + 1][kMaxCpus] = {};
+
+template <template <typename> class Atom>
+bool AccessSpreader<Atom>::initialized = AccessSpreader<Atom>::initialize();
+
+// Suppress this instantiation in other translation units. It is
+// instantiated in CacheLocality.cpp
+extern template struct AccessSpreader<std::atomic>;
+
+/**
+ * A simple freelist allocator. Allocates things of size sz, from
+ * slabs of size allocSize. Takes a lock on each
+ * allocation/deallocation.
+ */
+class SimpleAllocator {
+ std::mutex m_;
+ uint8_t* mem_{nullptr};
+ uint8_t* end_{nullptr};
+ void* freelist_{nullptr};
+ size_t allocSize_;
+ size_t sz_;
+ std::vector<void*> blocks_;
+
+ public:
+ SimpleAllocator(size_t allocSize, size_t sz);
+ ~SimpleAllocator();
+ void* allocateHard();
+
+ // Inline fast-paths.
+ void* allocate() {
+ std::lock_guard<std::mutex> g(m_);
+ // Freelist allocation.
+ if (freelist_) {
+ auto mem = freelist_;
+ freelist_ = *static_cast<void**>(freelist_);
+ return mem;
+ }
+
+ // Bump-ptr allocation.
+ if (intptr_t(mem_) % 128 == 0) {
+ // Avoid allocating pointers that may look like malloc
+ // pointers.
+ mem_ += std::min(sz_, alignof(std::max_align_t));
+ }
+ if (mem_ && (mem_ + sz_ <= end_)) {
+ auto mem = mem_;
+ mem_ += sz_;
+
+ assert(intptr_t(mem) % 128 != 0);
+ return mem;
+ }
+
+ return allocateHard();
+ }
+ void deallocate(void* mem) {
+ std::lock_guard<std::mutex> g(m_);
+ *static_cast<void**>(mem) = freelist_;
+ freelist_ = mem;
+ }
+};
+
+/**
+ * An allocator that can be used with CacheLocality to allocate
+ * core-local memory.
+ *
+ * There is actually nothing special about the memory itself (it is
+ * not bound to numa nodes or anything), but the allocator guarantees
+ * that memory allocatd from the same stripe will only come from cache
+ * lines also allocated to the same stripe. This means multiple
+ * things using CacheLocality can allocate memory in smaller-than
+ * cacheline increments, and be assured that it won't cause more false
+ * sharing than it otherwise would.
+ *
+ * Note that allocation and deallocation takes a per-sizeclass lock.
+ */
+template <size_t Stripes>
+class CoreAllocator {
+ public:
+ class Allocator {
+ static constexpr size_t AllocSize{4096};
+
+ uint8_t sizeClass(size_t size) {
+ if (size <= 8) {
+ return 0;
+ } else if (size <= 16) {
+ return 1;
+ } else if (size <= 32) {
+ return 2;
+ } else if (size <= 64) {
+ return 3;
+ } else { // punt to malloc.
+ return 4;
+ }
+ }
+
+ std::array<SimpleAllocator, 4> allocators_{
+ {{AllocSize, 8}, {AllocSize, 16}, {AllocSize, 32}, {AllocSize, 64}}};
+
+ public:
+ void* allocate(size_t size) {
+ auto cl = sizeClass(size);
+ if (cl == 4) {
+ static_assert(
+ CacheLocality::kFalseSharingRange == 128,
+ "kFalseSharingRange changed");
+ // Align to a cacheline
+ size = size + (CacheLocality::kFalseSharingRange - 1);
+ size &= ~size_t(CacheLocality::kFalseSharingRange - 1);
+ void* mem =
+ detail::aligned_malloc(size, CacheLocality::kFalseSharingRange);
+ if (!mem) {
+ std::__throw_bad_alloc();
+ }
+ return mem;
+ }
+ return allocators_[cl].allocate();
+ }
+ void deallocate(void* mem) {
+ if (!mem) {
+ return;
+ }
+
+ // See if it came from this allocator or malloc.
+ if (intptr_t(mem) % 128 != 0) {
+ auto addr =
+ reinterpret_cast<void*>(intptr_t(mem) & ~intptr_t(AllocSize - 1));
+ auto allocator = *static_cast<SimpleAllocator**>(addr);
+ allocator->deallocate(mem);
+ } else {
+ detail::aligned_free(mem);
+ }
+ }
+ };
+
+ Allocator* get(size_t stripe) {
+ assert(stripe < Stripes);
+ return &allocators_[stripe];
+ }
+
+ private:
+ Allocator allocators_[Stripes];
+};
+
+template <size_t Stripes>
+typename CoreAllocator<Stripes>::Allocator* getCoreAllocator(size_t stripe) {
+ // We cannot make sure that the allocator will be destroyed after
+ // all the objects allocated with it, so we leak it.
+ static Indestructible<CoreAllocator<Stripes>> allocator;
+ return allocator->get(stripe);
+}
+
+template <typename T, size_t Stripes>
+StlAllocator<typename CoreAllocator<Stripes>::Allocator, T> getCoreAllocatorStl(
+ size_t stripe) {
+ auto alloc = getCoreAllocator<Stripes>(stripe);
+ return StlAllocator<typename CoreAllocator<Stripes>::Allocator, T>(alloc);
+}
+
+} // namespace folly
#include <memory>
#include <folly/Enumerate.h>
-#include <folly/detail/CacheLocality.h>
+#include <folly/concurrency/CacheLocality.h>
namespace folly {
// prevent false sharing. Their control blocks will be adjacent
// thanks to allocate_shared().
for (auto slot : folly::enumerate(slots_)) {
- auto alloc = detail::getCoreAllocatorStl<Holder, kNumSlots>(slot.index);
+ auto alloc = getCoreAllocatorStl<Holder, kNumSlots>(slot.index);
auto holder = std::allocate_shared<Holder>(alloc, p);
*slot = std::shared_ptr<T>(holder, p.get());
}
}
std::shared_ptr<T> get() const {
- return slots_[detail::AccessSpreader<>::current(kNumSlots)];
+ return slots_[AccessSpreader<>::current(kNumSlots)];
}
private:
}
std::weak_ptr<T> get() const {
- return slots_[detail::AccessSpreader<>::current(kNumSlots)];
+ return slots_[AccessSpreader<>::current(kNumSlots)];
}
private:
--- /dev/null
+/*
+ * 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/concurrency/CacheLocality.h>
+
+#include <memory>
+#include <thread>
+#include <unordered_map>
+
+#include <glog/logging.h>
+
+#include <folly/Benchmark.h>
+
+using namespace folly;
+
+#define DECLARE_SPREADER_TAG(tag, locality, func) \
+ namespace { \
+ template <typename dummy> \
+ struct tag {}; \
+ } \
+ namespace folly { \
+ template <> \
+ const CacheLocality& CacheLocality::system<tag>() { \
+ static auto* inst = new CacheLocality(locality); \
+ return *inst; \
+ } \
+ template <> \
+ Getcpu::Func AccessSpreader<tag>::pickGetcpuFunc() { \
+ return func; \
+ } \
+ }
+
+DECLARE_SPREADER_TAG(
+ ThreadLocalTag,
+ CacheLocality::system<>(),
+ folly::FallbackGetcpu<SequentialThreadId<std::atomic>>::getcpu)
+DECLARE_SPREADER_TAG(
+ PthreadSelfTag,
+ CacheLocality::system<>(),
+ folly::FallbackGetcpu<HashingThreadId>::getcpu)
+
+BENCHMARK(AccessSpreaderUse, iters) {
+ for (unsigned long i = 0; i < iters; ++i) {
+ auto x = AccessSpreader<>::current(16);
+ folly::doNotOptimizeAway(x);
+ }
+}
+
+// Benchmark scores here reflect the time for 32 threads to perform an
+// atomic increment on a dual-socket E5-2660 @ 2.2Ghz. Surprisingly,
+// if we don't separate the counters onto unique 128 byte stripes the
+// 1_stripe and 2_stripe results are identical, even though the L3 is
+// claimed to have 64 byte cache lines.
+//
+// Getcpu refers to the vdso getcpu implementation. ThreadLocal refers
+// to execution using SequentialThreadId, the fallback if the vdso
+// getcpu isn't available. PthreadSelf hashes the value returned from
+// pthread_self() as a fallback-fallback for systems that don't have
+// thread-local support.
+//
+// At 16_stripe_0_work and 32_stripe_0_work there is only L1 traffic,
+// so since the stripe selection is 12 nanos the atomic increments in
+// the L1 is ~17 nanos. At width 8_stripe_0_work the line is expected
+// to ping-pong almost every operation, since the loops have the same
+// duration. Widths 4 and 2 have the same behavior, but each tour of the
+// cache line is 4 and 8 cores long, respectively. These all suggest a
+// lower bound of 60 nanos for intra-chip handoff and increment between
+// the L1s.
+//
+// With 420 nanos of busywork per contended increment, the system can
+// hide all of the latency of a tour of length 4, but not quite one of
+// length 8. I was a bit surprised at how much worse the non-striped
+// version got. It seems that the inter-chip traffic also interferes
+// with the L1-only localWork.load(). When the local work is doubled
+// to about 1 microsecond we see that the inter-chip contention is still
+// very important, but subdivisions on the same chip don't matter.
+//
+// sudo nice -n -20 buck-out/gen/folly/test/cache_locality_test
+// --benchmark --bm_min_iters=1000000
+// ============================================================================
+// folly/test/CacheLocalityTest.cpp relative time/iter iters/s
+// ============================================================================
+// AccessSpreaderUse 11.94ns 83.79M
+// ----------------------------------------------------------------------------
+// contentionAtWidthGetcpu(1_stripe_0_work) 985.75ns 1.01M
+// contentionAtWidthGetcpu(2_stripe_0_work) 424.02ns 2.36M
+// contentionAtWidthGetcpu(4_stripe_0_work) 190.13ns 5.26M
+// contentionAtWidthGetcpu(8_stripe_0_work) 91.86ns 10.89M
+// contentionAtWidthGetcpu(16_stripe_0_work) 29.31ns 34.12M
+// contentionAtWidthGetcpu(32_stripe_0_work) 29.53ns 33.86M
+// contentionAtWidthGetcpu(64_stripe_0_work) 29.93ns 33.41M
+// contentionAtWidthThreadLocal(2_stripe_0_work) 609.21ns 1.64M
+// contentionAtWidthThreadLocal(4_stripe_0_work) 303.60ns 3.29M
+// contentionAtWidthThreadLocal(8_stripe_0_work) 246.57ns 4.06M
+// contentionAtWidthThreadLocal(16_stripe_0_work) 154.84ns 6.46M
+// contentionAtWidthThreadLocal(32_stripe_0_work) 24.14ns 41.43M
+// contentionAtWidthThreadLocal(64_stripe_0_work) 23.95ns 41.75M
+// contentionAtWidthPthreadSelf(2_stripe_0_work) 722.01ns 1.39M
+// contentionAtWidthPthreadSelf(4_stripe_0_work) 501.56ns 1.99M
+// contentionAtWidthPthreadSelf(8_stripe_0_work) 474.58ns 2.11M
+// contentionAtWidthPthreadSelf(16_stripe_0_work) 300.90ns 3.32M
+// contentionAtWidthPthreadSelf(32_stripe_0_work) 175.77ns 5.69M
+// contentionAtWidthPthreadSelf(64_stripe_0_work) 174.88ns 5.72M
+// atomicIncrBaseline(local_incr_0_work) 16.81ns 59.51M
+// ----------------------------------------------------------------------------
+// contentionAtWidthGetcpu(1_stripe_500_work) 1.82us 549.97K
+// contentionAtWidthGetcpu(2_stripe_500_work) 533.71ns 1.87M
+// contentionAtWidthGetcpu(4_stripe_500_work) 424.64ns 2.35M
+// contentionAtWidthGetcpu(8_stripe_500_work) 451.85ns 2.21M
+// contentionAtWidthGetcpu(16_stripe_500_work) 425.54ns 2.35M
+// contentionAtWidthGetcpu(32_stripe_500_work) 501.66ns 1.99M
+// atomicIncrBaseline(local_incr_500_work) 438.46ns 2.28M
+// ----------------------------------------------------------------------------
+// contentionAtWidthGetcpu(1_stripe_1000_work) 1.88us 532.20K
+// contentionAtWidthGetcpu(2_stripe_1000_work) 824.62ns 1.21M
+// contentionAtWidthGetcpu(4_stripe_1000_work) 803.56ns 1.24M
+// contentionAtWidthGetcpu(8_stripe_1000_work) 926.65ns 1.08M
+// contentionAtWidthGetcpu(16_stripe_1000_work) 900.10ns 1.11M
+// contentionAtWidthGetcpu(32_stripe_1000_work) 890.75ns 1.12M
+// atomicIncrBaseline(local_incr_1000_work) 774.47ns 1.29M
+// ============================================================================
+template <template <typename> class Tag>
+static void contentionAtWidth(size_t iters, size_t stripes, size_t work) {
+ const size_t counterAlignment = 128;
+ const size_t numThreads = 32;
+
+ folly::BenchmarkSuspender braces;
+
+ std::atomic<size_t> ready(0);
+ std::atomic<bool> go(false);
+
+ // while in theory the cache line size is 64 bytes, experiments show
+ // that we get contention on 128 byte boundaries for Ivy Bridge. The
+ // extra indirection adds 1 or 2 nanos
+ assert(counterAlignment >= sizeof(std::atomic<size_t>));
+ std::vector<char> raw(counterAlignment * stripes);
+
+ // if we happen to be using the tlsRoundRobin, then sequentially
+ // assigning the thread identifiers is the unlikely best-case scenario.
+ // We don't want to unfairly benefit or penalize. Computing the exact
+ // maximum likelihood of the probability distributions is annoying, so
+ // I approximate as 2/5 of the ids that have no threads, 2/5 that have
+ // 1, 2/15 that have 2, and 1/15 that have 3. We accomplish this by
+ // wrapping back to slot 0 when we hit 1/15 and 1/5.
+
+ std::vector<std::thread> threads;
+ while (threads.size() < numThreads) {
+ threads.push_back(std::thread([&, iters, stripes, work]() {
+ auto counters = std::vector<std::atomic<size_t>*>(stripes);
+ for (size_t i = 0; i < stripes; ++i) {
+ counters[i] =
+ new (raw.data() + counterAlignment * i) std::atomic<size_t>();
+ }
+
+ ready++;
+ while (!go.load()) {
+ std::this_thread::yield();
+ }
+ std::atomic<int> localWork(0);
+ for (size_t i = iters; i > 0; --i) {
+ ++*(counters[AccessSpreader<Tag>::current(stripes)]);
+ for (size_t j = work; j > 0; --j) {
+ localWork.load();
+ }
+ }
+ }));
+
+ if (threads.size() == numThreads / 15 || threads.size() == numThreads / 5) {
+ // create a few dummy threads to wrap back around to 0 mod numCpus
+ for (size_t i = threads.size(); i != numThreads; ++i) {
+ std::thread([&]() { AccessSpreader<Tag>::current(stripes); }).join();
+ }
+ }
+ }
+
+ while (ready < numThreads) {
+ std::this_thread::yield();
+ }
+ braces.dismiss();
+ go = true;
+
+ for (auto& thr : threads) {
+ thr.join();
+ }
+}
+
+static void atomicIncrBaseline(size_t iters,
+ size_t work,
+ size_t numThreads = 32) {
+ folly::BenchmarkSuspender braces;
+
+ std::atomic<bool> go(false);
+
+ std::vector<std::thread> threads;
+ while (threads.size() < numThreads) {
+ threads.push_back(std::thread([&]() {
+ while (!go.load()) {
+ std::this_thread::yield();
+ }
+ std::atomic<size_t> localCounter(0);
+ std::atomic<int> localWork(0);
+ for (size_t i = iters; i > 0; --i) {
+ localCounter++;
+ for (size_t j = work; j > 0; --j) {
+ localWork.load();
+ }
+ }
+ }));
+ }
+
+ braces.dismiss();
+ go = true;
+
+ for (auto& thr : threads) {
+ thr.join();
+ }
+}
+
+static void contentionAtWidthGetcpu(size_t iters, size_t stripes, size_t work) {
+ contentionAtWidth<std::atomic>(iters, stripes, work);
+}
+
+static void contentionAtWidthThreadLocal(size_t iters,
+ size_t stripes,
+ size_t work) {
+ contentionAtWidth<ThreadLocalTag>(iters, stripes, work);
+}
+
+static void contentionAtWidthPthreadSelf(size_t iters,
+ size_t stripes,
+ size_t work) {
+ contentionAtWidth<PthreadSelfTag>(iters, stripes, work);
+}
+
+BENCHMARK_DRAW_LINE()
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 1_stripe_0_work, 1, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 2_stripe_0_work, 2, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 4_stripe_0_work, 4, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 8_stripe_0_work, 8, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 16_stripe_0_work, 16, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 32_stripe_0_work, 32, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 64_stripe_0_work, 64, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthThreadLocal, 2_stripe_0_work, 2, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthThreadLocal, 4_stripe_0_work, 4, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthThreadLocal, 8_stripe_0_work, 8, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthThreadLocal, 16_stripe_0_work, 16, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthThreadLocal, 32_stripe_0_work, 32, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthThreadLocal, 64_stripe_0_work, 64, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthPthreadSelf, 2_stripe_0_work, 2, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthPthreadSelf, 4_stripe_0_work, 4, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthPthreadSelf, 8_stripe_0_work, 8, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthPthreadSelf, 16_stripe_0_work, 16, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthPthreadSelf, 32_stripe_0_work, 32, 0)
+BENCHMARK_NAMED_PARAM(contentionAtWidthPthreadSelf, 64_stripe_0_work, 64, 0)
+BENCHMARK_NAMED_PARAM(atomicIncrBaseline, local_incr_0_work, 0)
+BENCHMARK_DRAW_LINE()
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 1_stripe_500_work, 1, 500)
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 2_stripe_500_work, 2, 500)
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 4_stripe_500_work, 4, 500)
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 8_stripe_500_work, 8, 500)
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 16_stripe_500_work, 16, 500)
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 32_stripe_500_work, 32, 500)
+BENCHMARK_NAMED_PARAM(atomicIncrBaseline, local_incr_500_work, 500)
+BENCHMARK_DRAW_LINE()
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 1_stripe_1000_work, 1, 1000)
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 2_stripe_1000_work, 2, 1000)
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 4_stripe_1000_work, 4, 1000)
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 8_stripe_1000_work, 8, 1000)
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 16_stripe_1000_work, 16, 1000)
+BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 32_stripe_1000_work, 32, 1000)
+BENCHMARK_NAMED_PARAM(atomicIncrBaseline, local_incr_1000_work, 1000)
+
+int main(int argc, char** argv) {
+ gflags::ParseCommandLineFlags(&argc, &argv, true);
+ folly::runBenchmarks();
+ return 0;
+}
--- /dev/null
+/*
+ * 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/concurrency/CacheLocality.h>
+
+#include <folly/portability/GTest.h>
+
+#include <memory>
+#include <thread>
+#include <type_traits>
+#include <unordered_map>
+#include <glog/logging.h>
+
+using namespace folly;
+
+/// This is the relevant nodes from a production box's sysfs tree. If you
+/// think this map is ugly you should see the version of this test that
+/// used a real directory tree. To reduce the chance of testing error
+/// I haven't tried to remove the common prefix
+static std::unordered_map<std::string, std::string> fakeSysfsTree = {
+ {"/sys/devices/system/cpu/cpu0/cache/index0/shared_cpu_list", "0,17"},
+ {"/sys/devices/system/cpu/cpu0/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu0/cache/index1/shared_cpu_list", "0,17"},
+ {"/sys/devices/system/cpu/cpu0/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu0/cache/index2/shared_cpu_list", "0,17"},
+ {"/sys/devices/system/cpu/cpu0/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu0/cache/index3/shared_cpu_list", "0-8,17-23"},
+ {"/sys/devices/system/cpu/cpu0/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu1/cache/index0/shared_cpu_list", "1,18"},
+ {"/sys/devices/system/cpu/cpu1/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu1/cache/index1/shared_cpu_list", "1,18"},
+ {"/sys/devices/system/cpu/cpu1/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu1/cache/index2/shared_cpu_list", "1,18"},
+ {"/sys/devices/system/cpu/cpu1/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu1/cache/index3/shared_cpu_list", "0-8,17-23"},
+ {"/sys/devices/system/cpu/cpu1/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu2/cache/index0/shared_cpu_list", "2,19"},
+ {"/sys/devices/system/cpu/cpu2/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu2/cache/index1/shared_cpu_list", "2,19"},
+ {"/sys/devices/system/cpu/cpu2/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu2/cache/index2/shared_cpu_list", "2,19"},
+ {"/sys/devices/system/cpu/cpu2/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu2/cache/index3/shared_cpu_list", "0-8,17-23"},
+ {"/sys/devices/system/cpu/cpu2/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu3/cache/index0/shared_cpu_list", "3,20"},
+ {"/sys/devices/system/cpu/cpu3/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu3/cache/index1/shared_cpu_list", "3,20"},
+ {"/sys/devices/system/cpu/cpu3/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu3/cache/index2/shared_cpu_list", "3,20"},
+ {"/sys/devices/system/cpu/cpu3/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu3/cache/index3/shared_cpu_list", "0-8,17-23"},
+ {"/sys/devices/system/cpu/cpu3/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu4/cache/index0/shared_cpu_list", "4,21"},
+ {"/sys/devices/system/cpu/cpu4/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu4/cache/index1/shared_cpu_list", "4,21"},
+ {"/sys/devices/system/cpu/cpu4/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu4/cache/index2/shared_cpu_list", "4,21"},
+ {"/sys/devices/system/cpu/cpu4/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu4/cache/index3/shared_cpu_list", "0-8,17-23"},
+ {"/sys/devices/system/cpu/cpu4/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu5/cache/index0/shared_cpu_list", "5-6"},
+ {"/sys/devices/system/cpu/cpu5/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu5/cache/index1/shared_cpu_list", "5-6"},
+ {"/sys/devices/system/cpu/cpu5/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu5/cache/index2/shared_cpu_list", "5-6"},
+ {"/sys/devices/system/cpu/cpu5/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu5/cache/index3/shared_cpu_list", "0-8,17-23"},
+ {"/sys/devices/system/cpu/cpu5/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu6/cache/index0/shared_cpu_list", "5-6"},
+ {"/sys/devices/system/cpu/cpu6/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu6/cache/index1/shared_cpu_list", "5-6"},
+ {"/sys/devices/system/cpu/cpu6/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu6/cache/index2/shared_cpu_list", "5-6"},
+ {"/sys/devices/system/cpu/cpu6/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu6/cache/index3/shared_cpu_list", "0-8,17-23"},
+ {"/sys/devices/system/cpu/cpu6/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu7/cache/index0/shared_cpu_list", "7,22"},
+ {"/sys/devices/system/cpu/cpu7/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu7/cache/index1/shared_cpu_list", "7,22"},
+ {"/sys/devices/system/cpu/cpu7/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu7/cache/index2/shared_cpu_list", "7,22"},
+ {"/sys/devices/system/cpu/cpu7/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu7/cache/index3/shared_cpu_list", "0-8,17-23"},
+ {"/sys/devices/system/cpu/cpu7/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu8/cache/index0/shared_cpu_list", "8,23"},
+ {"/sys/devices/system/cpu/cpu8/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu8/cache/index1/shared_cpu_list", "8,23"},
+ {"/sys/devices/system/cpu/cpu8/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu8/cache/index2/shared_cpu_list", "8,23"},
+ {"/sys/devices/system/cpu/cpu8/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu8/cache/index3/shared_cpu_list", "0-8,17-23"},
+ {"/sys/devices/system/cpu/cpu8/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu9/cache/index0/shared_cpu_list", "9,24"},
+ {"/sys/devices/system/cpu/cpu9/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu9/cache/index1/shared_cpu_list", "9,24"},
+ {"/sys/devices/system/cpu/cpu9/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu9/cache/index2/shared_cpu_list", "9,24"},
+ {"/sys/devices/system/cpu/cpu9/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu9/cache/index3/shared_cpu_list", "9-16,24-31"},
+ {"/sys/devices/system/cpu/cpu9/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu10/cache/index0/shared_cpu_list", "10,25"},
+ {"/sys/devices/system/cpu/cpu10/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu10/cache/index1/shared_cpu_list", "10,25"},
+ {"/sys/devices/system/cpu/cpu10/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu10/cache/index2/shared_cpu_list", "10,25"},
+ {"/sys/devices/system/cpu/cpu10/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu10/cache/index3/shared_cpu_list",
+ "9-16,24-31"},
+ {"/sys/devices/system/cpu/cpu10/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu11/cache/index0/shared_cpu_list", "11,26"},
+ {"/sys/devices/system/cpu/cpu11/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu11/cache/index1/shared_cpu_list", "11,26"},
+ {"/sys/devices/system/cpu/cpu11/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu11/cache/index2/shared_cpu_list", "11,26"},
+ {"/sys/devices/system/cpu/cpu11/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu11/cache/index3/shared_cpu_list",
+ "9-16,24-31"},
+ {"/sys/devices/system/cpu/cpu11/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu12/cache/index0/shared_cpu_list", "12,27"},
+ {"/sys/devices/system/cpu/cpu12/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu12/cache/index1/shared_cpu_list", "12,27"},
+ {"/sys/devices/system/cpu/cpu12/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu12/cache/index2/shared_cpu_list", "12,27"},
+ {"/sys/devices/system/cpu/cpu12/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu12/cache/index3/shared_cpu_list",
+ "9-16,24-31"},
+ {"/sys/devices/system/cpu/cpu12/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu13/cache/index0/shared_cpu_list", "13,28"},
+ {"/sys/devices/system/cpu/cpu13/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu13/cache/index1/shared_cpu_list", "13,28"},
+ {"/sys/devices/system/cpu/cpu13/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu13/cache/index2/shared_cpu_list", "13,28"},
+ {"/sys/devices/system/cpu/cpu13/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu13/cache/index3/shared_cpu_list",
+ "9-16,24-31"},
+ {"/sys/devices/system/cpu/cpu13/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu14/cache/index0/shared_cpu_list", "14,29"},
+ {"/sys/devices/system/cpu/cpu14/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu14/cache/index1/shared_cpu_list", "14,29"},
+ {"/sys/devices/system/cpu/cpu14/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu14/cache/index2/shared_cpu_list", "14,29"},
+ {"/sys/devices/system/cpu/cpu14/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu14/cache/index3/shared_cpu_list",
+ "9-16,24-31"},
+ {"/sys/devices/system/cpu/cpu14/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu15/cache/index0/shared_cpu_list", "15,30"},
+ {"/sys/devices/system/cpu/cpu15/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu15/cache/index1/shared_cpu_list", "15,30"},
+ {"/sys/devices/system/cpu/cpu15/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu15/cache/index2/shared_cpu_list", "15,30"},
+ {"/sys/devices/system/cpu/cpu15/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu15/cache/index3/shared_cpu_list",
+ "9-16,24-31"},
+ {"/sys/devices/system/cpu/cpu15/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu16/cache/index0/shared_cpu_list", "16,31"},
+ {"/sys/devices/system/cpu/cpu16/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu16/cache/index1/shared_cpu_list", "16,31"},
+ {"/sys/devices/system/cpu/cpu16/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu16/cache/index2/shared_cpu_list", "16,31"},
+ {"/sys/devices/system/cpu/cpu16/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu16/cache/index3/shared_cpu_list",
+ "9-16,24-31"},
+ {"/sys/devices/system/cpu/cpu16/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu17/cache/index0/shared_cpu_list", "0,17"},
+ {"/sys/devices/system/cpu/cpu17/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu17/cache/index1/shared_cpu_list", "0,17"},
+ {"/sys/devices/system/cpu/cpu17/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu17/cache/index2/shared_cpu_list", "0,17"},
+ {"/sys/devices/system/cpu/cpu17/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu17/cache/index3/shared_cpu_list", "0-8,17-23"},
+ {"/sys/devices/system/cpu/cpu17/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu18/cache/index0/shared_cpu_list", "1,18"},
+ {"/sys/devices/system/cpu/cpu18/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu18/cache/index1/shared_cpu_list", "1,18"},
+ {"/sys/devices/system/cpu/cpu18/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu18/cache/index2/shared_cpu_list", "1,18"},
+ {"/sys/devices/system/cpu/cpu18/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu18/cache/index3/shared_cpu_list", "0-8,17-23"},
+ {"/sys/devices/system/cpu/cpu18/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu19/cache/index0/shared_cpu_list", "2,19"},
+ {"/sys/devices/system/cpu/cpu19/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu19/cache/index1/shared_cpu_list", "2,19"},
+ {"/sys/devices/system/cpu/cpu19/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu19/cache/index2/shared_cpu_list", "2,19"},
+ {"/sys/devices/system/cpu/cpu19/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu19/cache/index3/shared_cpu_list", "0-8,17-23"},
+ {"/sys/devices/system/cpu/cpu19/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu20/cache/index0/shared_cpu_list", "3,20"},
+ {"/sys/devices/system/cpu/cpu20/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu20/cache/index1/shared_cpu_list", "3,20"},
+ {"/sys/devices/system/cpu/cpu20/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu20/cache/index2/shared_cpu_list", "3,20"},
+ {"/sys/devices/system/cpu/cpu20/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu20/cache/index3/shared_cpu_list", "0-8,17-23"},
+ {"/sys/devices/system/cpu/cpu20/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu21/cache/index0/shared_cpu_list", "4,21"},
+ {"/sys/devices/system/cpu/cpu21/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu21/cache/index1/shared_cpu_list", "4,21"},
+ {"/sys/devices/system/cpu/cpu21/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu21/cache/index2/shared_cpu_list", "4,21"},
+ {"/sys/devices/system/cpu/cpu21/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu21/cache/index3/shared_cpu_list", "0-8,17-23"},
+ {"/sys/devices/system/cpu/cpu21/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu22/cache/index0/shared_cpu_list", "7,22"},
+ {"/sys/devices/system/cpu/cpu22/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu22/cache/index1/shared_cpu_list", "7,22"},
+ {"/sys/devices/system/cpu/cpu22/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu22/cache/index2/shared_cpu_list", "7,22"},
+ {"/sys/devices/system/cpu/cpu22/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu22/cache/index3/shared_cpu_list", "0-8,17-23"},
+ {"/sys/devices/system/cpu/cpu22/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu23/cache/index0/shared_cpu_list", "8,23"},
+ {"/sys/devices/system/cpu/cpu23/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu23/cache/index1/shared_cpu_list", "8,23"},
+ {"/sys/devices/system/cpu/cpu23/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu23/cache/index2/shared_cpu_list", "8,23"},
+ {"/sys/devices/system/cpu/cpu23/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu23/cache/index3/shared_cpu_list", "0-8,17-23"},
+ {"/sys/devices/system/cpu/cpu23/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu24/cache/index0/shared_cpu_list", "9,24"},
+ {"/sys/devices/system/cpu/cpu24/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu24/cache/index1/shared_cpu_list", "9,24"},
+ {"/sys/devices/system/cpu/cpu24/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu24/cache/index2/shared_cpu_list", "9,24"},
+ {"/sys/devices/system/cpu/cpu24/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu24/cache/index3/shared_cpu_list",
+ "9-16,24-31"},
+ {"/sys/devices/system/cpu/cpu24/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu25/cache/index0/shared_cpu_list", "10,25"},
+ {"/sys/devices/system/cpu/cpu25/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu25/cache/index1/shared_cpu_list", "10,25"},
+ {"/sys/devices/system/cpu/cpu25/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu25/cache/index2/shared_cpu_list", "10,25"},
+ {"/sys/devices/system/cpu/cpu25/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu25/cache/index3/shared_cpu_list",
+ "9-16,24-31"},
+ {"/sys/devices/system/cpu/cpu25/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu26/cache/index0/shared_cpu_list", "11,26"},
+ {"/sys/devices/system/cpu/cpu26/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu26/cache/index1/shared_cpu_list", "11,26"},
+ {"/sys/devices/system/cpu/cpu26/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu26/cache/index2/shared_cpu_list", "11,26"},
+ {"/sys/devices/system/cpu/cpu26/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu26/cache/index3/shared_cpu_list",
+ "9-16,24-31"},
+ {"/sys/devices/system/cpu/cpu26/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu27/cache/index0/shared_cpu_list", "12,27"},
+ {"/sys/devices/system/cpu/cpu27/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu27/cache/index1/shared_cpu_list", "12,27"},
+ {"/sys/devices/system/cpu/cpu27/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu27/cache/index2/shared_cpu_list", "12,27"},
+ {"/sys/devices/system/cpu/cpu27/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu27/cache/index3/shared_cpu_list",
+ "9-16,24-31"},
+ {"/sys/devices/system/cpu/cpu27/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu28/cache/index0/shared_cpu_list", "13,28"},
+ {"/sys/devices/system/cpu/cpu28/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu28/cache/index1/shared_cpu_list", "13,28"},
+ {"/sys/devices/system/cpu/cpu28/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu28/cache/index2/shared_cpu_list", "13,28"},
+ {"/sys/devices/system/cpu/cpu28/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu28/cache/index3/shared_cpu_list",
+ "9-16,24-31"},
+ {"/sys/devices/system/cpu/cpu28/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu29/cache/index0/shared_cpu_list", "14,29"},
+ {"/sys/devices/system/cpu/cpu29/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu29/cache/index1/shared_cpu_list", "14,29"},
+ {"/sys/devices/system/cpu/cpu29/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu29/cache/index2/shared_cpu_list", "14,29"},
+ {"/sys/devices/system/cpu/cpu29/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu29/cache/index3/shared_cpu_list",
+ "9-16,24-31"},
+ {"/sys/devices/system/cpu/cpu29/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu30/cache/index0/shared_cpu_list", "15,30"},
+ {"/sys/devices/system/cpu/cpu30/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu30/cache/index1/shared_cpu_list", "15,30"},
+ {"/sys/devices/system/cpu/cpu30/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu30/cache/index2/shared_cpu_list", "15,30"},
+ {"/sys/devices/system/cpu/cpu30/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu30/cache/index3/shared_cpu_list",
+ "9-16,24-31"},
+ {"/sys/devices/system/cpu/cpu30/cache/index3/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu31/cache/index0/shared_cpu_list", "16,31"},
+ {"/sys/devices/system/cpu/cpu31/cache/index0/type", "Data"},
+ {"/sys/devices/system/cpu/cpu31/cache/index1/shared_cpu_list", "16,31"},
+ {"/sys/devices/system/cpu/cpu31/cache/index1/type", "Instruction"},
+ {"/sys/devices/system/cpu/cpu31/cache/index2/shared_cpu_list", "16,31"},
+ {"/sys/devices/system/cpu/cpu31/cache/index2/type", "Unified"},
+ {"/sys/devices/system/cpu/cpu31/cache/index3/shared_cpu_list",
+ "9-16,24-31"},
+ {"/sys/devices/system/cpu/cpu31/cache/index3/type", "Unified"}};
+
+/// This is the expected CacheLocality structure for fakeSysfsTree
+static const CacheLocality nonUniformExampleLocality = {32,
+ {16, 16, 2},
+ {0,
+ 2,
+ 4,
+ 6,
+ 8,
+ 10,
+ 11,
+ 12,
+ 14,
+ 16,
+ 18,
+ 20,
+ 22,
+ 24,
+ 26,
+ 28,
+ 30,
+ 1,
+ 3,
+ 5,
+ 7,
+ 9,
+ 13,
+ 15,
+ 17,
+ 19,
+ 21,
+ 23,
+ 25,
+ 27,
+ 29,
+ 31}};
+
+TEST(CacheLocality, FakeSysfs) {
+ auto parsed = CacheLocality::readFromSysfsTree([](std::string name) {
+ auto iter = fakeSysfsTree.find(name);
+ return iter == fakeSysfsTree.end() ? std::string() : iter->second;
+ });
+
+ auto& expected = nonUniformExampleLocality;
+ EXPECT_EQ(expected.numCpus, parsed.numCpus);
+ EXPECT_EQ(expected.numCachesByLevel, parsed.numCachesByLevel);
+ EXPECT_EQ(expected.localityIndexByCpu, parsed.localityIndexByCpu);
+}
+
+#if FOLLY_HAVE_LINUX_VDSO
+TEST(Getcpu, VdsoGetcpu) {
+ unsigned cpu;
+ Getcpu::resolveVdsoFunc()(&cpu, nullptr, nullptr);
+
+ EXPECT_TRUE(cpu < CPU_SETSIZE);
+}
+#endif
+
+#ifdef FOLLY_TLS
+TEST(ThreadId, SimpleTls) {
+ unsigned cpu = 0;
+ auto rv = folly::FallbackGetcpu<SequentialThreadId<std::atomic>>::getcpu(
+ &cpu, nullptr, nullptr);
+ EXPECT_EQ(rv, 0);
+ EXPECT_TRUE(cpu > 0);
+ unsigned again;
+ folly::FallbackGetcpu<SequentialThreadId<std::atomic>>::getcpu(
+ &again, nullptr, nullptr);
+ EXPECT_EQ(cpu, again);
+}
+#endif
+
+TEST(ThreadId, SimplePthread) {
+ unsigned cpu = 0;
+ auto rv =
+ folly::FallbackGetcpu<HashingThreadId>::getcpu(&cpu, nullptr, nullptr);
+ EXPECT_EQ(rv, 0);
+ EXPECT_TRUE(cpu > 0);
+ unsigned again;
+ folly::FallbackGetcpu<HashingThreadId>::getcpu(&again, nullptr, nullptr);
+ EXPECT_EQ(cpu, again);
+}
+
+#ifdef FOLLY_TLS
+static FOLLY_TLS unsigned testingCpu = 0;
+
+static int testingGetcpu(unsigned* cpu, unsigned* node, void* /* unused */) {
+ if (cpu != nullptr) {
+ *cpu = testingCpu;
+ }
+ if (node != nullptr) {
+ *node = testingCpu;
+ }
+ return 0;
+}
+#endif
+
+TEST(AccessSpreader, Simple) {
+ for (size_t s = 1; s < 200; ++s) {
+ EXPECT_LT(AccessSpreader<>::current(s), s);
+ }
+}
+
+#ifdef FOLLY_TLS
+#define DECLARE_SPREADER_TAG(tag, locality, func) \
+ namespace { \
+ template <typename dummy> \
+ struct tag {}; \
+ } \
+ namespace folly { \
+ template <> \
+ const CacheLocality& CacheLocality::system<tag>() { \
+ static auto* inst = new CacheLocality(locality); \
+ return *inst; \
+ } \
+ template <> \
+ Getcpu::Func AccessSpreader<tag>::pickGetcpuFunc() { \
+ return func; \
+ } \
+ }
+
+DECLARE_SPREADER_TAG(ManualTag, CacheLocality::uniform(16), testingGetcpu)
+
+TEST(AccessSpreader, Wrapping) {
+ // this test won't pass unless locality.numCpus divides kMaxCpus
+ auto numCpus = CacheLocality::system<ManualTag>().numCpus;
+ EXPECT_EQ(0, 128 % numCpus);
+ for (size_t s = 1; s < 200; ++s) {
+ for (size_t c = 0; c < 400; ++c) {
+ testingCpu = c;
+ auto observed = AccessSpreader<ManualTag>::current(s);
+ testingCpu = c % numCpus;
+ auto expected = AccessSpreader<ManualTag>::current(s);
+ EXPECT_EQ(expected, observed) << "numCpus=" << numCpus << ", s=" << s
+ << ", c=" << c;
+ }
+ }
+}
+
+TEST(CoreAllocator, Basic) {
+ CoreAllocator<32> alloc;
+ auto a = alloc.get(0);
+ auto res = a->allocate(8);
+ memset(res, 0, 8);
+ a->deallocate(res);
+ res = a->allocate(8);
+ EXPECT_TRUE((intptr_t)res % 8 == 0); // check alignment
+ memset(res, 0, 8);
+ a->deallocate(res);
+ res = a->allocate(12);
+ EXPECT_TRUE((intptr_t)res % 16 == 0); // check alignment
+ memset(res, 0, 12);
+ a->deallocate(res);
+ res = a->allocate(257);
+ memset(res, 0, 257);
+ a->deallocate(res);
+
+ std::vector<void*> mems;
+ for (int i = 0; i < 10000; i++) {
+ mems.push_back(a->allocate(1));
+ }
+ for (auto& mem : mems) {
+ a->deallocate(mem);
+ }
+ mems.clear();
+}
+
+#endif
+++ /dev/null
-/*
- * 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/detail/CacheLocality.h>
-
-#ifndef _MSC_VER
-#define _GNU_SOURCE 1 // for RTLD_NOLOAD
-#include <dlfcn.h>
-#endif
-#include <fstream>
-
-#include <folly/Conv.h>
-#include <folly/Exception.h>
-#include <folly/FileUtil.h>
-#include <folly/Format.h>
-#include <folly/ScopeGuard.h>
-
-namespace folly {
-namespace detail {
-
-///////////// CacheLocality
-
-/// Returns the best real CacheLocality information available
-static CacheLocality getSystemLocalityInfo() {
-#ifdef __linux__
- try {
- return CacheLocality::readFromSysfs();
- } catch (...) {
- // keep trying
- }
-#endif
-
- long numCpus = sysconf(_SC_NPROCESSORS_CONF);
- if (numCpus <= 0) {
- // This shouldn't happen, but if it does we should try to keep
- // going. We are probably not going to be able to parse /sys on
- // this box either (although we will try), which means we are going
- // to fall back to the SequentialThreadId splitter. On my 16 core
- // (x hyperthreading) dev box 16 stripes is enough to get pretty good
- // contention avoidance with SequentialThreadId, and there is little
- // improvement from going from 32 to 64. This default gives us some
- // wiggle room
- numCpus = 32;
- }
- return CacheLocality::uniform(size_t(numCpus));
-}
-
-template <>
-const CacheLocality& CacheLocality::system<std::atomic>() {
- static auto* cache = new CacheLocality(getSystemLocalityInfo());
- return *cache;
-}
-
-// Each level of cache has sharing sets, which are the set of cpus
-// that share a common cache at that level. These are available in a
-// hex bitset form (/sys/devices/system/cpu/cpu0/index0/shared_cpu_map,
-// for example). They are also available in a human-readable list form,
-// as in /sys/devices/system/cpu/cpu0/index0/shared_cpu_list. The list
-// is a comma-separated list of numbers and ranges, where the ranges are
-// a pair of decimal numbers separated by a '-'.
-//
-// To sort the cpus for optimum locality we don't really need to parse
-// the sharing sets, we just need a unique representative from the
-// equivalence class. The smallest value works fine, and happens to be
-// the first decimal number in the file. We load all of the equivalence
-// class information from all of the cpu*/index* directories, order the
-// cpus first by increasing last-level cache equivalence class, then by
-// the smaller caches. Finally, we break ties with the cpu number itself.
-
-/// Returns the first decimal number in the string, or throws an exception
-/// if the string does not start with a number terminated by ',', '-',
-/// '\n', or eos.
-static size_t parseLeadingNumber(const std::string& line) {
- auto raw = line.c_str();
- char* end;
- unsigned long val = strtoul(raw, &end, 10);
- if (end == raw || (*end != ',' && *end != '-' && *end != '\n' && *end != 0)) {
- throw std::runtime_error(
- to<std::string>("error parsing list '", line, "'").c_str());
- }
- return val;
-}
-
-CacheLocality CacheLocality::readFromSysfsTree(
- const std::function<std::string(std::string)>& mapping) {
- // number of equivalence classes per level
- std::vector<size_t> numCachesByLevel;
-
- // the list of cache equivalence classes, where equivalance classes
- // are named by the smallest cpu in the class
- std::vector<std::vector<size_t>> equivClassesByCpu;
-
- std::vector<size_t> cpus;
-
- while (true) {
- auto cpu = cpus.size();
- std::vector<size_t> levels;
- for (size_t index = 0;; ++index) {
- auto dir =
- sformat("/sys/devices/system/cpu/cpu{}/cache/index{}/", cpu, index);
- auto cacheType = mapping(dir + "type");
- auto equivStr = mapping(dir + "shared_cpu_list");
- if (cacheType.size() == 0 || equivStr.size() == 0) {
- // no more caches
- break;
- }
- if (cacheType[0] == 'I') {
- // cacheType in { "Data", "Instruction", "Unified" }. skip icache
- continue;
- }
- auto equiv = parseLeadingNumber(equivStr);
- auto level = levels.size();
- levels.push_back(equiv);
-
- if (equiv == cpu) {
- // we only want to count the equiv classes once, so we do it when
- // we first encounter them
- while (numCachesByLevel.size() <= level) {
- numCachesByLevel.push_back(0);
- }
- numCachesByLevel[level]++;
- }
- }
-
- if (levels.size() == 0) {
- // no levels at all for this cpu, we must be done
- break;
- }
- equivClassesByCpu.emplace_back(std::move(levels));
- cpus.push_back(cpu);
- }
-
- if (cpus.size() == 0) {
- throw std::runtime_error("unable to load cache sharing info");
- }
-
- std::sort(cpus.begin(),
- cpus.end(),
- [&](size_t lhs, size_t rhs) -> bool {
- // sort first by equiv class of cache with highest index,
- // direction doesn't matter. If different cpus have
- // different numbers of caches then this code might produce
- // a sub-optimal ordering, but it won't crash
- auto& lhsEquiv = equivClassesByCpu[lhs];
- auto& rhsEquiv = equivClassesByCpu[rhs];
- for (ssize_t i = ssize_t(std::min(lhsEquiv.size(), rhsEquiv.size())) - 1;
- i >= 0;
- --i) {
- auto idx = size_t(i);
- if (lhsEquiv[idx] != rhsEquiv[idx]) {
- return lhsEquiv[idx] < rhsEquiv[idx];
- }
- }
-
- // break ties deterministically by cpu
- return lhs < rhs;
- });
-
- // the cpus are now sorted by locality, with neighboring entries closer
- // to each other than entries that are far away. For striping we want
- // the inverse map, since we are starting with the cpu
- std::vector<size_t> indexes(cpus.size());
- for (size_t i = 0; i < cpus.size(); ++i) {
- indexes[cpus[i]] = i;
- }
-
- return CacheLocality{
- cpus.size(), std::move(numCachesByLevel), std::move(indexes)};
-}
-
-CacheLocality CacheLocality::readFromSysfs() {
- return readFromSysfsTree([](std::string name) {
- std::ifstream xi(name.c_str());
- std::string rv;
- std::getline(xi, rv);
- return rv;
- });
-}
-
-CacheLocality CacheLocality::uniform(size_t numCpus) {
- CacheLocality rv;
-
- rv.numCpus = numCpus;
-
- // one cache shared by all cpus
- rv.numCachesByLevel.push_back(numCpus);
-
- // no permutations in locality index mapping
- for (size_t cpu = 0; cpu < numCpus; ++cpu) {
- rv.localityIndexByCpu.push_back(cpu);
- }
-
- return rv;
-}
-
-////////////// Getcpu
-
-Getcpu::Func Getcpu::resolveVdsoFunc() {
-#if !FOLLY_HAVE_LINUX_VDSO
- return nullptr;
-#else
- void* h = dlopen("linux-vdso.so.1", RTLD_LAZY | RTLD_LOCAL | RTLD_NOLOAD);
- if (h == nullptr) {
- return nullptr;
- }
-
- auto func = Getcpu::Func(dlsym(h, "__vdso_getcpu"));
- if (func == nullptr) {
- // technically a null result could either be a failure or a successful
- // lookup of a symbol with the null value, but the second can't actually
- // happen for this symbol. No point holding the handle forever if
- // we don't need the code
- dlclose(h);
- }
-
- return func;
-#endif
-}
-
-#ifdef FOLLY_TLS
-/////////////// SequentialThreadId
-template struct SequentialThreadId<std::atomic>;
-#endif
-
-/////////////// AccessSpreader
-template struct AccessSpreader<std::atomic>;
-
-SimpleAllocator::SimpleAllocator(size_t allocSize, size_t sz)
- : allocSize_{allocSize}, sz_(sz) {}
-
-SimpleAllocator::~SimpleAllocator() {
- std::lock_guard<std::mutex> g(m_);
- for (auto& block : blocks_) {
- aligned_free(block);
- }
-}
-
-void* SimpleAllocator::allocateHard() {
- // Allocate a new slab.
- mem_ = static_cast<uint8_t*>(aligned_malloc(allocSize_, allocSize_));
- if (!mem_) {
- std::__throw_bad_alloc();
- }
- end_ = mem_ + allocSize_;
- blocks_.push_back(mem_);
-
- // Install a pointer to ourselves as the allocator.
- *reinterpret_cast<SimpleAllocator**>(mem_) = this;
- static_assert(
- alignof(std::max_align_t) >= sizeof(SimpleAllocator*),
- "alignment too small");
- mem_ += std::min(sz_, alignof(std::max_align_t));
-
- // New allocation.
- auto mem = mem_;
- mem_ += sz_;
- assert(intptr_t(mem) % 128 != 0);
- return mem;
-}
-
-} // namespace detail
-} // namespace folly
+++ /dev/null
-/*
- * 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 <algorithm>
-#include <array>
-#include <atomic>
-#include <cassert>
-#include <functional>
-#include <limits>
-#include <mutex>
-#include <string>
-#include <type_traits>
-#include <unordered_map>
-#include <vector>
-
-#include <folly/Hash.h>
-#include <folly/Indestructible.h>
-#include <folly/Likely.h>
-#include <folly/Memory.h>
-#include <folly/Portability.h>
-#include <folly/ThreadId.h>
-#include <folly/portability/BitsFunctexcept.h>
-#include <folly/portability/Memory.h>
-
-namespace folly {
-namespace detail {
-
-// This file contains several classes that might be useful if you are
-// trying to dynamically optimize cache locality: CacheLocality reads
-// cache sharing information from sysfs to determine how CPUs should be
-// grouped to minimize contention, Getcpu provides fast access to the
-// current CPU via __vdso_getcpu, and AccessSpreader uses these two to
-// optimally spread accesses among a predetermined number of stripes.
-//
-// AccessSpreader<>::current(n) microbenchmarks at 22 nanos, which is
-// substantially less than the cost of a cache miss. This means that we
-// can effectively use it to reduce cache line ping-pong on striped data
-// structures such as IndexedMemPool or statistics counters.
-//
-// Because CacheLocality looks at all of the cache levels, it can be
-// used for different levels of optimization. AccessSpreader(2) does
-// per-chip spreading on a dual socket system. AccessSpreader(numCpus)
-// does perfect per-cpu spreading. AccessSpreader(numCpus / 2) does
-// perfect L1 spreading in a system with hyperthreading enabled.
-
-struct CacheLocality {
-
- /// 1 more than the maximum value that can be returned from sched_getcpu
- /// or getcpu. This is the number of hardware thread contexts provided
- /// by the processors
- size_t numCpus;
-
- /// Holds the number of caches present at each cache level (0 is
- /// the closest to the cpu). This is the number of AccessSpreader
- /// stripes needed to avoid cross-cache communication at the specified
- /// layer. numCachesByLevel.front() is the number of L1 caches and
- /// numCachesByLevel.back() is the number of last-level caches.
- std::vector<size_t> numCachesByLevel;
-
- /// A map from cpu (from sched_getcpu or getcpu) to an index in the
- /// range 0..numCpus-1, where neighboring locality indices are more
- /// likely to share caches then indices far away. All of the members
- /// of a particular cache level be contiguous in their locality index.
- /// For example, if numCpus is 32 and numCachesByLevel.back() is 2,
- /// then cpus with a locality index < 16 will share one last-level
- /// cache and cpus with a locality index >= 16 will share the other.
- std::vector<size_t> localityIndexByCpu;
-
- /// Returns the best CacheLocality information available for the current
- /// system, cached for fast access. This will be loaded from sysfs if
- /// possible, otherwise it will be correct in the number of CPUs but
- /// not in their sharing structure.
- ///
- /// If you are into yo dawgs, this is a shared cache of the local
- /// locality of the shared caches.
- ///
- /// The template parameter here is used to allow injection of a
- /// repeatable CacheLocality structure during testing. Rather than
- /// inject the type of the CacheLocality provider into every data type
- /// that transitively uses it, all components select between the default
- /// sysfs implementation and a deterministic implementation by keying
- /// off the type of the underlying atomic. See DeterministicScheduler.
- template <template <typename> class Atom = std::atomic>
- static const CacheLocality& system();
-
- /// Reads CacheLocality information from a tree structured like
- /// the sysfs filesystem. The provided function will be evaluated
- /// for each sysfs file that needs to be queried. The function
- /// should return a string containing the first line of the file
- /// (not including the newline), or an empty string if the file does
- /// not exist. The function will be called with paths of the form
- /// /sys/devices/system/cpu/cpu*/cache/index*/{type,shared_cpu_list} .
- /// Throws an exception if no caches can be parsed at all.
- static CacheLocality readFromSysfsTree(
- const std::function<std::string(std::string)>& mapping);
-
- /// Reads CacheLocality information from the real sysfs filesystem.
- /// Throws an exception if no cache information can be loaded.
- static CacheLocality readFromSysfs();
-
- /// Returns a usable (but probably not reflective of reality)
- /// CacheLocality structure with the specified number of cpus and a
- /// single cache level that associates one cpu per cache.
- static CacheLocality uniform(size_t numCpus);
-
- enum {
- /// Memory locations on the same cache line are subject to false
- /// sharing, which is very bad for performance. Microbenchmarks
- /// indicate that pairs of cache lines also see interference under
- /// heavy use of atomic operations (observed for atomic increment on
- /// Sandy Bridge). See FOLLY_ALIGN_TO_AVOID_FALSE_SHARING
- kFalseSharingRange = 128
- };
-
- static_assert(
- kFalseSharingRange == 128,
- "FOLLY_ALIGN_TO_AVOID_FALSE_SHARING should track kFalseSharingRange");
-};
-
-// TODO replace __attribute__ with alignas and 128 with kFalseSharingRange
-
-/// An attribute that will cause a variable or field to be aligned so that
-/// it doesn't have false sharing with anything at a smaller memory address.
-#define FOLLY_ALIGN_TO_AVOID_FALSE_SHARING FOLLY_ALIGNED(128)
-
-/// Knows how to derive a function pointer to the VDSO implementation of
-/// getcpu(2), if available
-struct Getcpu {
- /// Function pointer to a function with the same signature as getcpu(2).
- typedef int (*Func)(unsigned* cpu, unsigned* node, void* unused);
-
- /// Returns a pointer to the VDSO implementation of getcpu(2), if
- /// available, or nullptr otherwise. This function may be quite
- /// expensive, be sure to cache the result.
- static Func resolveVdsoFunc();
-};
-
-#ifdef FOLLY_TLS
-template <template <typename> class Atom>
-struct SequentialThreadId {
-
- /// Returns the thread id assigned to the current thread
- static unsigned get() {
- auto rv = currentId;
- if (UNLIKELY(rv == 0)) {
- rv = currentId = ++prevId;
- }
- return rv;
- }
-
- private:
- static Atom<unsigned> prevId;
-
- static FOLLY_TLS unsigned currentId;
-};
-
-template <template <typename> class Atom>
-Atom<unsigned> SequentialThreadId<Atom>::prevId(0);
-
-template <template <typename> class Atom>
-FOLLY_TLS unsigned SequentialThreadId<Atom>::currentId(0);
-
-// Suppress this instantiation in other translation units. It is
-// instantiated in CacheLocality.cpp
-extern template struct SequentialThreadId<std::atomic>;
-#endif
-
-struct HashingThreadId {
- static unsigned get() {
- return hash::twang_32from64(getCurrentThreadID());
- }
-};
-
-/// A class that lazily binds a unique (for each implementation of Atom)
-/// identifier to a thread. This is a fallback mechanism for the access
-/// spreader if __vdso_getcpu can't be loaded
-template <typename ThreadId>
-struct FallbackGetcpu {
- /// Fills the thread id into the cpu and node out params (if they
- /// are non-null). This method is intended to act like getcpu when a
- /// fast-enough form of getcpu isn't available or isn't desired
- static int getcpu(unsigned* cpu, unsigned* node, void* /* unused */) {
- auto id = ThreadId::get();
- if (cpu) {
- *cpu = id;
- }
- if (node) {
- *node = id;
- }
- return 0;
- }
-};
-
-#ifdef FOLLY_TLS
-typedef FallbackGetcpu<SequentialThreadId<std::atomic>> FallbackGetcpuType;
-#else
-typedef FallbackGetcpu<HashingThreadId> FallbackGetcpuType;
-#endif
-
-/// AccessSpreader arranges access to a striped data structure in such a
-/// way that concurrently executing threads are likely to be accessing
-/// different stripes. It does NOT guarantee uncontended access.
-/// Your underlying algorithm must be thread-safe without spreading, this
-/// is merely an optimization. AccessSpreader::current(n) is typically
-/// much faster than a cache miss (12 nanos on my dev box, tested fast
-/// in both 2.6 and 3.2 kernels).
-///
-/// If available (and not using the deterministic testing implementation)
-/// AccessSpreader uses the getcpu system call via VDSO and the
-/// precise locality information retrieved from sysfs by CacheLocality.
-/// This provides optimal anti-sharing at a fraction of the cost of a
-/// cache miss.
-///
-/// When there are not as many stripes as processors, we try to optimally
-/// place the cache sharing boundaries. This means that if you have 2
-/// stripes and run on a dual-socket system, your 2 stripes will each get
-/// all of the cores from a single socket. If you have 16 stripes on a
-/// 16 core system plus hyperthreading (32 cpus), each core will get its
-/// own stripe and there will be no cache sharing at all.
-///
-/// AccessSpreader has a fallback mechanism for when __vdso_getcpu can't be
-/// loaded, or for use during deterministic testing. Using sched_getcpu
-/// or the getcpu syscall would negate the performance advantages of
-/// access spreading, so we use a thread-local value and a shared atomic
-/// counter to spread access out. On systems lacking both a fast getcpu()
-/// and TLS, we hash the thread id to spread accesses.
-///
-/// AccessSpreader is templated on the template type that is used
-/// to implement atomics, as a way to instantiate the underlying
-/// heuristics differently for production use and deterministic unit
-/// testing. See DeterministicScheduler for more. If you aren't using
-/// DeterministicScheduler, you can just use the default template parameter
-/// all of the time.
-template <template <typename> class Atom = std::atomic>
-struct AccessSpreader {
-
- /// Returns the stripe associated with the current CPU. The returned
- /// value will be < numStripes.
- static size_t current(size_t numStripes) {
- // widthAndCpuToStripe[0] will actually work okay (all zeros), but
- // something's wrong with the caller
- assert(numStripes > 0);
-
- unsigned cpu;
- getcpuFunc(&cpu, nullptr, nullptr);
- return widthAndCpuToStripe[std::min(size_t(kMaxCpus),
- numStripes)][cpu % kMaxCpus];
- }
-
- private:
- /// If there are more cpus than this nothing will crash, but there
- /// might be unnecessary sharing
- enum { kMaxCpus = 128 };
-
- typedef uint8_t CompactStripe;
-
- static_assert((kMaxCpus & (kMaxCpus - 1)) == 0,
- "kMaxCpus should be a power of two so modulo is fast");
- static_assert(kMaxCpus - 1 <= std::numeric_limits<CompactStripe>::max(),
- "stripeByCpu element type isn't wide enough");
-
- /// Points to the getcpu-like function we are using to obtain the
- /// current cpu. It should not be assumed that the returned cpu value
- /// is in range. We use a static for this so that we can prearrange a
- /// valid value in the pre-constructed state and avoid the need for a
- /// conditional on every subsequent invocation (not normally a big win,
- /// but 20% on some inner loops here).
- static Getcpu::Func getcpuFunc;
-
- /// For each level of splitting up to kMaxCpus, maps the cpu (mod
- /// kMaxCpus) to the stripe. Rather than performing any inequalities
- /// or modulo on the actual number of cpus, we just fill in the entire
- /// array.
- static CompactStripe widthAndCpuToStripe[kMaxCpus + 1][kMaxCpus];
-
- static bool initialized;
-
- /// Returns the best getcpu implementation for Atom
- static Getcpu::Func pickGetcpuFunc() {
- auto best = Getcpu::resolveVdsoFunc();
- return best ? best : &FallbackGetcpuType::getcpu;
- }
-
- /// Always claims to be on CPU zero, node zero
- static int degenerateGetcpu(unsigned* cpu, unsigned* node, void*) {
- if (cpu != nullptr) {
- *cpu = 0;
- }
- if (node != nullptr) {
- *node = 0;
- }
- return 0;
- }
-
- // The function to call for fast lookup of getcpu is a singleton, as
- // is the precomputed table of locality information. AccessSpreader
- // is used in very tight loops, however (we're trying to race an L1
- // cache miss!), so the normal singleton mechanisms are noticeably
- // expensive. Even a not-taken branch guarding access to getcpuFunc
- // slows AccessSpreader::current from 12 nanos to 14. As a result, we
- // populate the static members with simple (but valid) values that can
- // be filled in by the linker, and then follow up with a normal static
- // initializer call that puts in the proper version. This means that
- // when there are initialization order issues we will just observe a
- // zero stripe. Once a sanitizer gets smart enough to detect this as
- // a race or undefined behavior, we can annotate it.
-
- static bool initialize() {
- getcpuFunc = pickGetcpuFunc();
-
- auto& cacheLocality = CacheLocality::system<Atom>();
- auto n = cacheLocality.numCpus;
- for (size_t width = 0; width <= kMaxCpus; ++width) {
- auto numStripes = std::max(size_t{1}, width);
- for (size_t cpu = 0; cpu < kMaxCpus && cpu < n; ++cpu) {
- auto index = cacheLocality.localityIndexByCpu[cpu];
- assert(index < n);
- // as index goes from 0..n, post-transform value goes from
- // 0..numStripes
- widthAndCpuToStripe[width][cpu] =
- CompactStripe((index * numStripes) / n);
- assert(widthAndCpuToStripe[width][cpu] < numStripes);
- }
- for (size_t cpu = n; cpu < kMaxCpus; ++cpu) {
- widthAndCpuToStripe[width][cpu] = widthAndCpuToStripe[width][cpu - n];
- }
- }
- return true;
- }
-};
-
-template <template <typename> class Atom>
-Getcpu::Func AccessSpreader<Atom>::getcpuFunc =
- AccessSpreader<Atom>::degenerateGetcpu;
-
-template <template <typename> class Atom>
-typename AccessSpreader<Atom>::CompactStripe
- AccessSpreader<Atom>::widthAndCpuToStripe[kMaxCpus + 1][kMaxCpus] = {};
-
-template <template <typename> class Atom>
-bool AccessSpreader<Atom>::initialized = AccessSpreader<Atom>::initialize();
-
-// Suppress this instantiation in other translation units. It is
-// instantiated in CacheLocality.cpp
-extern template struct AccessSpreader<std::atomic>;
-
-/**
- * A simple freelist allocator. Allocates things of size sz, from
- * slabs of size allocSize. Takes a lock on each
- * allocation/deallocation.
- */
-class SimpleAllocator {
- std::mutex m_;
- uint8_t* mem_{nullptr};
- uint8_t* end_{nullptr};
- void* freelist_{nullptr};
- size_t allocSize_;
- size_t sz_;
- std::vector<void*> blocks_;
-
- public:
- SimpleAllocator(size_t allocSize, size_t sz);
- ~SimpleAllocator();
- void* allocateHard();
-
- // Inline fast-paths.
- void* allocate() {
- std::lock_guard<std::mutex> g(m_);
- // Freelist allocation.
- if (freelist_) {
- auto mem = freelist_;
- freelist_ = *static_cast<void**>(freelist_);
- return mem;
- }
-
- // Bump-ptr allocation.
- if (intptr_t(mem_) % 128 == 0) {
- // Avoid allocating pointers that may look like malloc
- // pointers.
- mem_ += std::min(sz_, alignof(std::max_align_t));
- }
- if (mem_ && (mem_ + sz_ <= end_)) {
- auto mem = mem_;
- mem_ += sz_;
-
- assert(intptr_t(mem) % 128 != 0);
- return mem;
- }
-
- return allocateHard();
- }
- void deallocate(void* mem) {
- std::lock_guard<std::mutex> g(m_);
- *static_cast<void**>(mem) = freelist_;
- freelist_ = mem;
- }
-};
-
-/**
- * An allocator that can be used with CacheLocality to allocate
- * core-local memory.
- *
- * There is actually nothing special about the memory itself (it is
- * not bound to numa nodes or anything), but the allocator guarantees
- * that memory allocatd from the same stripe will only come from cache
- * lines also allocated to the same stripe. This means multiple
- * things using CacheLocality can allocate memory in smaller-than
- * cacheline increments, and be assured that it won't cause more false
- * sharing than it otherwise would.
- *
- * Note that allocation and deallocation takes a per-sizeclass lock.
- */
-template <size_t Stripes>
-class CoreAllocator {
- public:
- class Allocator {
- static constexpr size_t AllocSize{4096};
-
- uint8_t sizeClass(size_t size) {
- if (size <= 8) {
- return 0;
- } else if (size <= 16) {
- return 1;
- } else if (size <= 32) {
- return 2;
- } else if (size <= 64) {
- return 3;
- } else { // punt to malloc.
- return 4;
- }
- }
-
- std::array<SimpleAllocator, 4> allocators_{
- {{AllocSize, 8}, {AllocSize, 16}, {AllocSize, 32}, {AllocSize, 64}}};
-
- public:
- void* allocate(size_t size) {
- auto cl = sizeClass(size);
- if (cl == 4) {
- static_assert(
- CacheLocality::kFalseSharingRange == 128,
- "kFalseSharingRange changed");
- // Align to a cacheline
- size = size + (CacheLocality::kFalseSharingRange - 1);
- size &= ~size_t(CacheLocality::kFalseSharingRange - 1);
- void* mem = aligned_malloc(size, CacheLocality::kFalseSharingRange);
- if (!mem) {
- std::__throw_bad_alloc();
- }
- return mem;
- }
- return allocators_[cl].allocate();
- }
- void deallocate(void* mem) {
- if (!mem) {
- return;
- }
-
- // See if it came from this allocator or malloc.
- if (intptr_t(mem) % 128 != 0) {
- auto addr =
- reinterpret_cast<void*>(intptr_t(mem) & ~intptr_t(AllocSize - 1));
- auto allocator = *static_cast<SimpleAllocator**>(addr);
- allocator->deallocate(mem);
- } else {
- aligned_free(mem);
- }
- }
- };
-
- Allocator* get(size_t stripe) {
- assert(stripe < Stripes);
- return &allocators_[stripe];
- }
-
- private:
- Allocator allocators_[Stripes];
-};
-
-template <size_t Stripes>
-typename CoreAllocator<Stripes>::Allocator* getCoreAllocator(size_t stripe) {
- // We cannot make sure that the allocator will be destroyed after
- // all the objects allocated with it, so we leak it.
- static Indestructible<CoreAllocator<Stripes>> allocator;
- return allocator->get(stripe);
-}
-
-template <typename T, size_t Stripes>
-StlAllocator<typename CoreAllocator<Stripes>::Allocator, T> getCoreAllocatorStl(
- size_t stripe) {
- auto alloc = getCoreAllocator<Stripes>(stripe);
- return StlAllocator<typename CoreAllocator<Stripes>::Allocator, T>(alloc);
-}
-
-} // namespace detail
-} // namespace folly
#pragma once
-#include <folly/detail/CacheLocality.h>
+#include <folly/concurrency/CacheLocality.h>
namespace folly {
// We need alignas(T) alignas(kFalseSharingRange) for the case where alignof(T)
// > alignof(kFalseSharingRange).
template <typename T>
-struct alignas(T) alignas(detail::CacheLocality::kFalseSharingRange)
+struct alignas(T) alignas(CacheLocality::kFalseSharingRange)
CachelinePaddedImpl<T, /* needsPadding = */ false> {
template <typename... Args>
explicit CachelinePaddedImpl(Args&&... args)
};
template <typename T>
-struct alignas(T) alignas(detail::CacheLocality::kFalseSharingRange)
+struct alignas(T) alignas(CacheLocality::kFalseSharingRange)
CachelinePaddedImpl<T, /* needsPadding = */ true> {
template <typename... Args>
explicit CachelinePaddedImpl(Args&&... args)
#include <folly/Malloc.h>
#include <folly/Portability.h>
#include <folly/ScopeGuard.h>
-#include <folly/detail/CacheLocality.h>
+#include <folly/concurrency/CacheLocality.h>
#include <folly/portability/PThread.h>
#include <folly/portability/SysMman.h>
#include <folly/portability/Unistd.h>
#include <folly/Function.h>
#include <folly/IndexedMemPool.h>
#include <folly/Portability.h>
-#include <folly/detail/CacheLocality.h>
+#include <folly/concurrency/CacheLocality.h>
#include <atomic>
#include <cassert>
+++ /dev/null
-/*
- * 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/detail/CacheLocality.h>
-
-#include <memory>
-#include <thread>
-#include <unordered_map>
-
-#include <glog/logging.h>
-
-#include <folly/Benchmark.h>
-
-using namespace folly::detail;
-
-#define DECLARE_SPREADER_TAG(tag, locality, func) \
- namespace { \
- template <typename dummy> \
- struct tag {}; \
- } \
- namespace folly { \
- namespace detail { \
- template <> \
- const CacheLocality& CacheLocality::system<tag>() { \
- static auto* inst = new CacheLocality(locality); \
- return *inst; \
- } \
- template <> \
- Getcpu::Func AccessSpreader<tag>::pickGetcpuFunc() { \
- return func; \
- } \
- } \
- }
-
-DECLARE_SPREADER_TAG(
- ThreadLocalTag,
- CacheLocality::system<>(),
- folly::detail::FallbackGetcpu<SequentialThreadId<std::atomic>>::getcpu)
-DECLARE_SPREADER_TAG(PthreadSelfTag,
- CacheLocality::system<>(),
- folly::detail::FallbackGetcpu<HashingThreadId>::getcpu)
-
-BENCHMARK(AccessSpreaderUse, iters) {
- for (unsigned long i = 0; i < iters; ++i) {
- auto x = AccessSpreader<>::current(16);
- folly::doNotOptimizeAway(x);
- }
-}
-
-// Benchmark scores here reflect the time for 32 threads to perform an
-// atomic increment on a dual-socket E5-2660 @ 2.2Ghz. Surprisingly,
-// if we don't separate the counters onto unique 128 byte stripes the
-// 1_stripe and 2_stripe results are identical, even though the L3 is
-// claimed to have 64 byte cache lines.
-//
-// Getcpu refers to the vdso getcpu implementation. ThreadLocal refers
-// to execution using SequentialThreadId, the fallback if the vdso
-// getcpu isn't available. PthreadSelf hashes the value returned from
-// pthread_self() as a fallback-fallback for systems that don't have
-// thread-local support.
-//
-// At 16_stripe_0_work and 32_stripe_0_work there is only L1 traffic,
-// so since the stripe selection is 12 nanos the atomic increments in
-// the L1 is ~17 nanos. At width 8_stripe_0_work the line is expected
-// to ping-pong almost every operation, since the loops have the same
-// duration. Widths 4 and 2 have the same behavior, but each tour of the
-// cache line is 4 and 8 cores long, respectively. These all suggest a
-// lower bound of 60 nanos for intra-chip handoff and increment between
-// the L1s.
-//
-// With 420 nanos of busywork per contended increment, the system can
-// hide all of the latency of a tour of length 4, but not quite one of
-// length 8. I was a bit surprised at how much worse the non-striped
-// version got. It seems that the inter-chip traffic also interferes
-// with the L1-only localWork.load(). When the local work is doubled
-// to about 1 microsecond we see that the inter-chip contention is still
-// very important, but subdivisions on the same chip don't matter.
-//
-// sudo nice -n -20 buck-out/gen/folly/test/cache_locality_test
-// --benchmark --bm_min_iters=1000000
-// ============================================================================
-// folly/test/CacheLocalityTest.cpp relative time/iter iters/s
-// ============================================================================
-// AccessSpreaderUse 11.94ns 83.79M
-// ----------------------------------------------------------------------------
-// contentionAtWidthGetcpu(1_stripe_0_work) 985.75ns 1.01M
-// contentionAtWidthGetcpu(2_stripe_0_work) 424.02ns 2.36M
-// contentionAtWidthGetcpu(4_stripe_0_work) 190.13ns 5.26M
-// contentionAtWidthGetcpu(8_stripe_0_work) 91.86ns 10.89M
-// contentionAtWidthGetcpu(16_stripe_0_work) 29.31ns 34.12M
-// contentionAtWidthGetcpu(32_stripe_0_work) 29.53ns 33.86M
-// contentionAtWidthGetcpu(64_stripe_0_work) 29.93ns 33.41M
-// contentionAtWidthThreadLocal(2_stripe_0_work) 609.21ns 1.64M
-// contentionAtWidthThreadLocal(4_stripe_0_work) 303.60ns 3.29M
-// contentionAtWidthThreadLocal(8_stripe_0_work) 246.57ns 4.06M
-// contentionAtWidthThreadLocal(16_stripe_0_work) 154.84ns 6.46M
-// contentionAtWidthThreadLocal(32_stripe_0_work) 24.14ns 41.43M
-// contentionAtWidthThreadLocal(64_stripe_0_work) 23.95ns 41.75M
-// contentionAtWidthPthreadSelf(2_stripe_0_work) 722.01ns 1.39M
-// contentionAtWidthPthreadSelf(4_stripe_0_work) 501.56ns 1.99M
-// contentionAtWidthPthreadSelf(8_stripe_0_work) 474.58ns 2.11M
-// contentionAtWidthPthreadSelf(16_stripe_0_work) 300.90ns 3.32M
-// contentionAtWidthPthreadSelf(32_stripe_0_work) 175.77ns 5.69M
-// contentionAtWidthPthreadSelf(64_stripe_0_work) 174.88ns 5.72M
-// atomicIncrBaseline(local_incr_0_work) 16.81ns 59.51M
-// ----------------------------------------------------------------------------
-// contentionAtWidthGetcpu(1_stripe_500_work) 1.82us 549.97K
-// contentionAtWidthGetcpu(2_stripe_500_work) 533.71ns 1.87M
-// contentionAtWidthGetcpu(4_stripe_500_work) 424.64ns 2.35M
-// contentionAtWidthGetcpu(8_stripe_500_work) 451.85ns 2.21M
-// contentionAtWidthGetcpu(16_stripe_500_work) 425.54ns 2.35M
-// contentionAtWidthGetcpu(32_stripe_500_work) 501.66ns 1.99M
-// atomicIncrBaseline(local_incr_500_work) 438.46ns 2.28M
-// ----------------------------------------------------------------------------
-// contentionAtWidthGetcpu(1_stripe_1000_work) 1.88us 532.20K
-// contentionAtWidthGetcpu(2_stripe_1000_work) 824.62ns 1.21M
-// contentionAtWidthGetcpu(4_stripe_1000_work) 803.56ns 1.24M
-// contentionAtWidthGetcpu(8_stripe_1000_work) 926.65ns 1.08M
-// contentionAtWidthGetcpu(16_stripe_1000_work) 900.10ns 1.11M
-// contentionAtWidthGetcpu(32_stripe_1000_work) 890.75ns 1.12M
-// atomicIncrBaseline(local_incr_1000_work) 774.47ns 1.29M
-// ============================================================================
-template <template <typename> class Tag>
-static void contentionAtWidth(size_t iters, size_t stripes, size_t work) {
- const size_t counterAlignment = 128;
- const size_t numThreads = 32;
-
- folly::BenchmarkSuspender braces;
-
- std::atomic<size_t> ready(0);
- std::atomic<bool> go(false);
-
- // while in theory the cache line size is 64 bytes, experiments show
- // that we get contention on 128 byte boundaries for Ivy Bridge. The
- // extra indirection adds 1 or 2 nanos
- assert(counterAlignment >= sizeof(std::atomic<size_t>));
- std::vector<char> raw(counterAlignment * stripes);
-
- // if we happen to be using the tlsRoundRobin, then sequentially
- // assigning the thread identifiers is the unlikely best-case scenario.
- // We don't want to unfairly benefit or penalize. Computing the exact
- // maximum likelihood of the probability distributions is annoying, so
- // I approximate as 2/5 of the ids that have no threads, 2/5 that have
- // 1, 2/15 that have 2, and 1/15 that have 3. We accomplish this by
- // wrapping back to slot 0 when we hit 1/15 and 1/5.
-
- std::vector<std::thread> threads;
- while (threads.size() < numThreads) {
- threads.push_back(std::thread([&, iters, stripes, work]() {
- auto counters = std::vector<std::atomic<size_t>*>(stripes);
- for (size_t i = 0; i < stripes; ++i) {
- counters[i] =
- new (raw.data() + counterAlignment * i) std::atomic<size_t>();
- }
-
- ready++;
- while (!go.load()) {
- std::this_thread::yield();
- }
- std::atomic<int> localWork(0);
- for (size_t i = iters; i > 0; --i) {
- ++*(counters[AccessSpreader<Tag>::current(stripes)]);
- for (size_t j = work; j > 0; --j) {
- localWork.load();
- }
- }
- }));
-
- if (threads.size() == numThreads / 15 || threads.size() == numThreads / 5) {
- // create a few dummy threads to wrap back around to 0 mod numCpus
- for (size_t i = threads.size(); i != numThreads; ++i) {
- std::thread([&]() { AccessSpreader<Tag>::current(stripes); }).join();
- }
- }
- }
-
- while (ready < numThreads) {
- std::this_thread::yield();
- }
- braces.dismiss();
- go = true;
-
- for (auto& thr : threads) {
- thr.join();
- }
-}
-
-static void atomicIncrBaseline(size_t iters,
- size_t work,
- size_t numThreads = 32) {
- folly::BenchmarkSuspender braces;
-
- std::atomic<bool> go(false);
-
- std::vector<std::thread> threads;
- while (threads.size() < numThreads) {
- threads.push_back(std::thread([&]() {
- while (!go.load()) {
- std::this_thread::yield();
- }
- std::atomic<size_t> localCounter(0);
- std::atomic<int> localWork(0);
- for (size_t i = iters; i > 0; --i) {
- localCounter++;
- for (size_t j = work; j > 0; --j) {
- localWork.load();
- }
- }
- }));
- }
-
- braces.dismiss();
- go = true;
-
- for (auto& thr : threads) {
- thr.join();
- }
-}
-
-static void contentionAtWidthGetcpu(size_t iters, size_t stripes, size_t work) {
- contentionAtWidth<std::atomic>(iters, stripes, work);
-}
-
-static void contentionAtWidthThreadLocal(size_t iters,
- size_t stripes,
- size_t work) {
- contentionAtWidth<ThreadLocalTag>(iters, stripes, work);
-}
-
-static void contentionAtWidthPthreadSelf(size_t iters,
- size_t stripes,
- size_t work) {
- contentionAtWidth<PthreadSelfTag>(iters, stripes, work);
-}
-
-BENCHMARK_DRAW_LINE()
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 1_stripe_0_work, 1, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 2_stripe_0_work, 2, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 4_stripe_0_work, 4, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 8_stripe_0_work, 8, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 16_stripe_0_work, 16, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 32_stripe_0_work, 32, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 64_stripe_0_work, 64, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthThreadLocal, 2_stripe_0_work, 2, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthThreadLocal, 4_stripe_0_work, 4, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthThreadLocal, 8_stripe_0_work, 8, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthThreadLocal, 16_stripe_0_work, 16, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthThreadLocal, 32_stripe_0_work, 32, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthThreadLocal, 64_stripe_0_work, 64, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthPthreadSelf, 2_stripe_0_work, 2, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthPthreadSelf, 4_stripe_0_work, 4, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthPthreadSelf, 8_stripe_0_work, 8, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthPthreadSelf, 16_stripe_0_work, 16, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthPthreadSelf, 32_stripe_0_work, 32, 0)
-BENCHMARK_NAMED_PARAM(contentionAtWidthPthreadSelf, 64_stripe_0_work, 64, 0)
-BENCHMARK_NAMED_PARAM(atomicIncrBaseline, local_incr_0_work, 0)
-BENCHMARK_DRAW_LINE()
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 1_stripe_500_work, 1, 500)
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 2_stripe_500_work, 2, 500)
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 4_stripe_500_work, 4, 500)
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 8_stripe_500_work, 8, 500)
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 16_stripe_500_work, 16, 500)
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 32_stripe_500_work, 32, 500)
-BENCHMARK_NAMED_PARAM(atomicIncrBaseline, local_incr_500_work, 500)
-BENCHMARK_DRAW_LINE()
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 1_stripe_1000_work, 1, 1000)
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 2_stripe_1000_work, 2, 1000)
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 4_stripe_1000_work, 4, 1000)
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 8_stripe_1000_work, 8, 1000)
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 16_stripe_1000_work, 16, 1000)
-BENCHMARK_NAMED_PARAM(contentionAtWidthGetcpu, 32_stripe_1000_work, 32, 1000)
-BENCHMARK_NAMED_PARAM(atomicIncrBaseline, local_incr_1000_work, 1000)
-
-int main(int argc, char** argv) {
- gflags::ParseCommandLineFlags(&argc, &argv, true);
- folly::runBenchmarks();
- return 0;
-}
+++ /dev/null
-/*
- * 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/detail/CacheLocality.h>
-
-#include <folly/portability/GTest.h>
-
-#include <memory>
-#include <thread>
-#include <type_traits>
-#include <unordered_map>
-#include <glog/logging.h>
-
-using namespace folly::detail;
-
-/// This is the relevant nodes from a production box's sysfs tree. If you
-/// think this map is ugly you should see the version of this test that
-/// used a real directory tree. To reduce the chance of testing error
-/// I haven't tried to remove the common prefix
-static std::unordered_map<std::string, std::string> fakeSysfsTree = {
- {"/sys/devices/system/cpu/cpu0/cache/index0/shared_cpu_list", "0,17"},
- {"/sys/devices/system/cpu/cpu0/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu0/cache/index1/shared_cpu_list", "0,17"},
- {"/sys/devices/system/cpu/cpu0/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu0/cache/index2/shared_cpu_list", "0,17"},
- {"/sys/devices/system/cpu/cpu0/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu0/cache/index3/shared_cpu_list", "0-8,17-23"},
- {"/sys/devices/system/cpu/cpu0/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu1/cache/index0/shared_cpu_list", "1,18"},
- {"/sys/devices/system/cpu/cpu1/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu1/cache/index1/shared_cpu_list", "1,18"},
- {"/sys/devices/system/cpu/cpu1/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu1/cache/index2/shared_cpu_list", "1,18"},
- {"/sys/devices/system/cpu/cpu1/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu1/cache/index3/shared_cpu_list", "0-8,17-23"},
- {"/sys/devices/system/cpu/cpu1/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu2/cache/index0/shared_cpu_list", "2,19"},
- {"/sys/devices/system/cpu/cpu2/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu2/cache/index1/shared_cpu_list", "2,19"},
- {"/sys/devices/system/cpu/cpu2/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu2/cache/index2/shared_cpu_list", "2,19"},
- {"/sys/devices/system/cpu/cpu2/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu2/cache/index3/shared_cpu_list", "0-8,17-23"},
- {"/sys/devices/system/cpu/cpu2/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu3/cache/index0/shared_cpu_list", "3,20"},
- {"/sys/devices/system/cpu/cpu3/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu3/cache/index1/shared_cpu_list", "3,20"},
- {"/sys/devices/system/cpu/cpu3/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu3/cache/index2/shared_cpu_list", "3,20"},
- {"/sys/devices/system/cpu/cpu3/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu3/cache/index3/shared_cpu_list", "0-8,17-23"},
- {"/sys/devices/system/cpu/cpu3/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu4/cache/index0/shared_cpu_list", "4,21"},
- {"/sys/devices/system/cpu/cpu4/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu4/cache/index1/shared_cpu_list", "4,21"},
- {"/sys/devices/system/cpu/cpu4/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu4/cache/index2/shared_cpu_list", "4,21"},
- {"/sys/devices/system/cpu/cpu4/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu4/cache/index3/shared_cpu_list", "0-8,17-23"},
- {"/sys/devices/system/cpu/cpu4/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu5/cache/index0/shared_cpu_list", "5-6"},
- {"/sys/devices/system/cpu/cpu5/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu5/cache/index1/shared_cpu_list", "5-6"},
- {"/sys/devices/system/cpu/cpu5/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu5/cache/index2/shared_cpu_list", "5-6"},
- {"/sys/devices/system/cpu/cpu5/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu5/cache/index3/shared_cpu_list", "0-8,17-23"},
- {"/sys/devices/system/cpu/cpu5/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu6/cache/index0/shared_cpu_list", "5-6"},
- {"/sys/devices/system/cpu/cpu6/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu6/cache/index1/shared_cpu_list", "5-6"},
- {"/sys/devices/system/cpu/cpu6/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu6/cache/index2/shared_cpu_list", "5-6"},
- {"/sys/devices/system/cpu/cpu6/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu6/cache/index3/shared_cpu_list", "0-8,17-23"},
- {"/sys/devices/system/cpu/cpu6/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu7/cache/index0/shared_cpu_list", "7,22"},
- {"/sys/devices/system/cpu/cpu7/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu7/cache/index1/shared_cpu_list", "7,22"},
- {"/sys/devices/system/cpu/cpu7/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu7/cache/index2/shared_cpu_list", "7,22"},
- {"/sys/devices/system/cpu/cpu7/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu7/cache/index3/shared_cpu_list", "0-8,17-23"},
- {"/sys/devices/system/cpu/cpu7/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu8/cache/index0/shared_cpu_list", "8,23"},
- {"/sys/devices/system/cpu/cpu8/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu8/cache/index1/shared_cpu_list", "8,23"},
- {"/sys/devices/system/cpu/cpu8/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu8/cache/index2/shared_cpu_list", "8,23"},
- {"/sys/devices/system/cpu/cpu8/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu8/cache/index3/shared_cpu_list", "0-8,17-23"},
- {"/sys/devices/system/cpu/cpu8/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu9/cache/index0/shared_cpu_list", "9,24"},
- {"/sys/devices/system/cpu/cpu9/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu9/cache/index1/shared_cpu_list", "9,24"},
- {"/sys/devices/system/cpu/cpu9/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu9/cache/index2/shared_cpu_list", "9,24"},
- {"/sys/devices/system/cpu/cpu9/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu9/cache/index3/shared_cpu_list", "9-16,24-31"},
- {"/sys/devices/system/cpu/cpu9/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu10/cache/index0/shared_cpu_list", "10,25"},
- {"/sys/devices/system/cpu/cpu10/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu10/cache/index1/shared_cpu_list", "10,25"},
- {"/sys/devices/system/cpu/cpu10/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu10/cache/index2/shared_cpu_list", "10,25"},
- {"/sys/devices/system/cpu/cpu10/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu10/cache/index3/shared_cpu_list",
- "9-16,24-31"},
- {"/sys/devices/system/cpu/cpu10/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu11/cache/index0/shared_cpu_list", "11,26"},
- {"/sys/devices/system/cpu/cpu11/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu11/cache/index1/shared_cpu_list", "11,26"},
- {"/sys/devices/system/cpu/cpu11/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu11/cache/index2/shared_cpu_list", "11,26"},
- {"/sys/devices/system/cpu/cpu11/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu11/cache/index3/shared_cpu_list",
- "9-16,24-31"},
- {"/sys/devices/system/cpu/cpu11/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu12/cache/index0/shared_cpu_list", "12,27"},
- {"/sys/devices/system/cpu/cpu12/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu12/cache/index1/shared_cpu_list", "12,27"},
- {"/sys/devices/system/cpu/cpu12/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu12/cache/index2/shared_cpu_list", "12,27"},
- {"/sys/devices/system/cpu/cpu12/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu12/cache/index3/shared_cpu_list",
- "9-16,24-31"},
- {"/sys/devices/system/cpu/cpu12/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu13/cache/index0/shared_cpu_list", "13,28"},
- {"/sys/devices/system/cpu/cpu13/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu13/cache/index1/shared_cpu_list", "13,28"},
- {"/sys/devices/system/cpu/cpu13/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu13/cache/index2/shared_cpu_list", "13,28"},
- {"/sys/devices/system/cpu/cpu13/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu13/cache/index3/shared_cpu_list",
- "9-16,24-31"},
- {"/sys/devices/system/cpu/cpu13/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu14/cache/index0/shared_cpu_list", "14,29"},
- {"/sys/devices/system/cpu/cpu14/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu14/cache/index1/shared_cpu_list", "14,29"},
- {"/sys/devices/system/cpu/cpu14/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu14/cache/index2/shared_cpu_list", "14,29"},
- {"/sys/devices/system/cpu/cpu14/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu14/cache/index3/shared_cpu_list",
- "9-16,24-31"},
- {"/sys/devices/system/cpu/cpu14/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu15/cache/index0/shared_cpu_list", "15,30"},
- {"/sys/devices/system/cpu/cpu15/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu15/cache/index1/shared_cpu_list", "15,30"},
- {"/sys/devices/system/cpu/cpu15/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu15/cache/index2/shared_cpu_list", "15,30"},
- {"/sys/devices/system/cpu/cpu15/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu15/cache/index3/shared_cpu_list",
- "9-16,24-31"},
- {"/sys/devices/system/cpu/cpu15/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu16/cache/index0/shared_cpu_list", "16,31"},
- {"/sys/devices/system/cpu/cpu16/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu16/cache/index1/shared_cpu_list", "16,31"},
- {"/sys/devices/system/cpu/cpu16/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu16/cache/index2/shared_cpu_list", "16,31"},
- {"/sys/devices/system/cpu/cpu16/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu16/cache/index3/shared_cpu_list",
- "9-16,24-31"},
- {"/sys/devices/system/cpu/cpu16/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu17/cache/index0/shared_cpu_list", "0,17"},
- {"/sys/devices/system/cpu/cpu17/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu17/cache/index1/shared_cpu_list", "0,17"},
- {"/sys/devices/system/cpu/cpu17/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu17/cache/index2/shared_cpu_list", "0,17"},
- {"/sys/devices/system/cpu/cpu17/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu17/cache/index3/shared_cpu_list", "0-8,17-23"},
- {"/sys/devices/system/cpu/cpu17/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu18/cache/index0/shared_cpu_list", "1,18"},
- {"/sys/devices/system/cpu/cpu18/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu18/cache/index1/shared_cpu_list", "1,18"},
- {"/sys/devices/system/cpu/cpu18/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu18/cache/index2/shared_cpu_list", "1,18"},
- {"/sys/devices/system/cpu/cpu18/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu18/cache/index3/shared_cpu_list", "0-8,17-23"},
- {"/sys/devices/system/cpu/cpu18/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu19/cache/index0/shared_cpu_list", "2,19"},
- {"/sys/devices/system/cpu/cpu19/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu19/cache/index1/shared_cpu_list", "2,19"},
- {"/sys/devices/system/cpu/cpu19/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu19/cache/index2/shared_cpu_list", "2,19"},
- {"/sys/devices/system/cpu/cpu19/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu19/cache/index3/shared_cpu_list", "0-8,17-23"},
- {"/sys/devices/system/cpu/cpu19/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu20/cache/index0/shared_cpu_list", "3,20"},
- {"/sys/devices/system/cpu/cpu20/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu20/cache/index1/shared_cpu_list", "3,20"},
- {"/sys/devices/system/cpu/cpu20/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu20/cache/index2/shared_cpu_list", "3,20"},
- {"/sys/devices/system/cpu/cpu20/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu20/cache/index3/shared_cpu_list", "0-8,17-23"},
- {"/sys/devices/system/cpu/cpu20/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu21/cache/index0/shared_cpu_list", "4,21"},
- {"/sys/devices/system/cpu/cpu21/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu21/cache/index1/shared_cpu_list", "4,21"},
- {"/sys/devices/system/cpu/cpu21/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu21/cache/index2/shared_cpu_list", "4,21"},
- {"/sys/devices/system/cpu/cpu21/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu21/cache/index3/shared_cpu_list", "0-8,17-23"},
- {"/sys/devices/system/cpu/cpu21/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu22/cache/index0/shared_cpu_list", "7,22"},
- {"/sys/devices/system/cpu/cpu22/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu22/cache/index1/shared_cpu_list", "7,22"},
- {"/sys/devices/system/cpu/cpu22/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu22/cache/index2/shared_cpu_list", "7,22"},
- {"/sys/devices/system/cpu/cpu22/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu22/cache/index3/shared_cpu_list", "0-8,17-23"},
- {"/sys/devices/system/cpu/cpu22/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu23/cache/index0/shared_cpu_list", "8,23"},
- {"/sys/devices/system/cpu/cpu23/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu23/cache/index1/shared_cpu_list", "8,23"},
- {"/sys/devices/system/cpu/cpu23/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu23/cache/index2/shared_cpu_list", "8,23"},
- {"/sys/devices/system/cpu/cpu23/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu23/cache/index3/shared_cpu_list", "0-8,17-23"},
- {"/sys/devices/system/cpu/cpu23/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu24/cache/index0/shared_cpu_list", "9,24"},
- {"/sys/devices/system/cpu/cpu24/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu24/cache/index1/shared_cpu_list", "9,24"},
- {"/sys/devices/system/cpu/cpu24/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu24/cache/index2/shared_cpu_list", "9,24"},
- {"/sys/devices/system/cpu/cpu24/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu24/cache/index3/shared_cpu_list",
- "9-16,24-31"},
- {"/sys/devices/system/cpu/cpu24/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu25/cache/index0/shared_cpu_list", "10,25"},
- {"/sys/devices/system/cpu/cpu25/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu25/cache/index1/shared_cpu_list", "10,25"},
- {"/sys/devices/system/cpu/cpu25/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu25/cache/index2/shared_cpu_list", "10,25"},
- {"/sys/devices/system/cpu/cpu25/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu25/cache/index3/shared_cpu_list",
- "9-16,24-31"},
- {"/sys/devices/system/cpu/cpu25/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu26/cache/index0/shared_cpu_list", "11,26"},
- {"/sys/devices/system/cpu/cpu26/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu26/cache/index1/shared_cpu_list", "11,26"},
- {"/sys/devices/system/cpu/cpu26/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu26/cache/index2/shared_cpu_list", "11,26"},
- {"/sys/devices/system/cpu/cpu26/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu26/cache/index3/shared_cpu_list",
- "9-16,24-31"},
- {"/sys/devices/system/cpu/cpu26/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu27/cache/index0/shared_cpu_list", "12,27"},
- {"/sys/devices/system/cpu/cpu27/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu27/cache/index1/shared_cpu_list", "12,27"},
- {"/sys/devices/system/cpu/cpu27/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu27/cache/index2/shared_cpu_list", "12,27"},
- {"/sys/devices/system/cpu/cpu27/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu27/cache/index3/shared_cpu_list",
- "9-16,24-31"},
- {"/sys/devices/system/cpu/cpu27/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu28/cache/index0/shared_cpu_list", "13,28"},
- {"/sys/devices/system/cpu/cpu28/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu28/cache/index1/shared_cpu_list", "13,28"},
- {"/sys/devices/system/cpu/cpu28/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu28/cache/index2/shared_cpu_list", "13,28"},
- {"/sys/devices/system/cpu/cpu28/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu28/cache/index3/shared_cpu_list",
- "9-16,24-31"},
- {"/sys/devices/system/cpu/cpu28/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu29/cache/index0/shared_cpu_list", "14,29"},
- {"/sys/devices/system/cpu/cpu29/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu29/cache/index1/shared_cpu_list", "14,29"},
- {"/sys/devices/system/cpu/cpu29/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu29/cache/index2/shared_cpu_list", "14,29"},
- {"/sys/devices/system/cpu/cpu29/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu29/cache/index3/shared_cpu_list",
- "9-16,24-31"},
- {"/sys/devices/system/cpu/cpu29/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu30/cache/index0/shared_cpu_list", "15,30"},
- {"/sys/devices/system/cpu/cpu30/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu30/cache/index1/shared_cpu_list", "15,30"},
- {"/sys/devices/system/cpu/cpu30/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu30/cache/index2/shared_cpu_list", "15,30"},
- {"/sys/devices/system/cpu/cpu30/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu30/cache/index3/shared_cpu_list",
- "9-16,24-31"},
- {"/sys/devices/system/cpu/cpu30/cache/index3/type", "Unified"},
- {"/sys/devices/system/cpu/cpu31/cache/index0/shared_cpu_list", "16,31"},
- {"/sys/devices/system/cpu/cpu31/cache/index0/type", "Data"},
- {"/sys/devices/system/cpu/cpu31/cache/index1/shared_cpu_list", "16,31"},
- {"/sys/devices/system/cpu/cpu31/cache/index1/type", "Instruction"},
- {"/sys/devices/system/cpu/cpu31/cache/index2/shared_cpu_list", "16,31"},
- {"/sys/devices/system/cpu/cpu31/cache/index2/type", "Unified"},
- {"/sys/devices/system/cpu/cpu31/cache/index3/shared_cpu_list",
- "9-16,24-31"},
- {"/sys/devices/system/cpu/cpu31/cache/index3/type", "Unified"}};
-
-/// This is the expected CacheLocality structure for fakeSysfsTree
-static const CacheLocality nonUniformExampleLocality = {32,
- {16, 16, 2},
- {0,
- 2,
- 4,
- 6,
- 8,
- 10,
- 11,
- 12,
- 14,
- 16,
- 18,
- 20,
- 22,
- 24,
- 26,
- 28,
- 30,
- 1,
- 3,
- 5,
- 7,
- 9,
- 13,
- 15,
- 17,
- 19,
- 21,
- 23,
- 25,
- 27,
- 29,
- 31}};
-
-TEST(CacheLocality, FakeSysfs) {
- auto parsed = CacheLocality::readFromSysfsTree([](std::string name) {
- auto iter = fakeSysfsTree.find(name);
- return iter == fakeSysfsTree.end() ? std::string() : iter->second;
- });
-
- auto& expected = nonUniformExampleLocality;
- EXPECT_EQ(expected.numCpus, parsed.numCpus);
- EXPECT_EQ(expected.numCachesByLevel, parsed.numCachesByLevel);
- EXPECT_EQ(expected.localityIndexByCpu, parsed.localityIndexByCpu);
-}
-
-#if FOLLY_HAVE_LINUX_VDSO
-TEST(Getcpu, VdsoGetcpu) {
- unsigned cpu;
- Getcpu::resolveVdsoFunc()(&cpu, nullptr, nullptr);
-
- EXPECT_TRUE(cpu < CPU_SETSIZE);
-}
-#endif
-
-#ifdef FOLLY_TLS
-TEST(ThreadId, SimpleTls) {
- unsigned cpu = 0;
- auto rv =
- folly::detail::FallbackGetcpu<SequentialThreadId<std::atomic>>::getcpu(
- &cpu, nullptr, nullptr);
- EXPECT_EQ(rv, 0);
- EXPECT_TRUE(cpu > 0);
- unsigned again;
- folly::detail::FallbackGetcpu<SequentialThreadId<std::atomic>>::getcpu(
- &again, nullptr, nullptr);
- EXPECT_EQ(cpu, again);
-}
-#endif
-
-TEST(ThreadId, SimplePthread) {
- unsigned cpu = 0;
- auto rv = folly::detail::FallbackGetcpu<HashingThreadId>::getcpu(
- &cpu, nullptr, nullptr);
- EXPECT_EQ(rv, 0);
- EXPECT_TRUE(cpu > 0);
- unsigned again;
- folly::detail::FallbackGetcpu<HashingThreadId>::getcpu(
- &again, nullptr, nullptr);
- EXPECT_EQ(cpu, again);
-}
-
-#ifdef FOLLY_TLS
-static FOLLY_TLS unsigned testingCpu = 0;
-
-static int testingGetcpu(unsigned* cpu, unsigned* node, void* /* unused */) {
- if (cpu != nullptr) {
- *cpu = testingCpu;
- }
- if (node != nullptr) {
- *node = testingCpu;
- }
- return 0;
-}
-#endif
-
-TEST(AccessSpreader, Simple) {
- for (size_t s = 1; s < 200; ++s) {
- EXPECT_LT(AccessSpreader<>::current(s), s);
- }
-}
-
-#ifdef FOLLY_TLS
-#define DECLARE_SPREADER_TAG(tag, locality, func) \
- namespace { \
- template <typename dummy> \
- struct tag {}; \
- } \
- namespace folly { \
- namespace detail { \
- template <> \
- const CacheLocality& CacheLocality::system<tag>() { \
- static auto* inst = new CacheLocality(locality); \
- return *inst; \
- } \
- template <> \
- Getcpu::Func AccessSpreader<tag>::pickGetcpuFunc() { \
- return func; \
- } \
- } \
- }
-
-DECLARE_SPREADER_TAG(ManualTag, CacheLocality::uniform(16), testingGetcpu)
-
-TEST(AccessSpreader, Wrapping) {
- // this test won't pass unless locality.numCpus divides kMaxCpus
- auto numCpus = CacheLocality::system<ManualTag>().numCpus;
- EXPECT_EQ(0, 128 % numCpus);
- for (size_t s = 1; s < 200; ++s) {
- for (size_t c = 0; c < 400; ++c) {
- testingCpu = c;
- auto observed = AccessSpreader<ManualTag>::current(s);
- testingCpu = c % numCpus;
- auto expected = AccessSpreader<ManualTag>::current(s);
- EXPECT_EQ(expected, observed) << "numCpus=" << numCpus << ", s=" << s
- << ", c=" << c;
- }
- }
-}
-
-TEST(CoreAllocator, Basic) {
- CoreAllocator<32> alloc;
- auto a = alloc.get(0);
- auto res = a->allocate(8);
- memset(res, 0, 8);
- a->deallocate(res);
- res = a->allocate(8);
- EXPECT_TRUE((intptr_t)res % 8 == 0); // check alignment
- memset(res, 0, 8);
- a->deallocate(res);
- res = a->allocate(12);
- EXPECT_TRUE((intptr_t)res % 16 == 0); // check alignment
- memset(res, 0, 12);
- a->deallocate(res);
- res = a->allocate(257);
- memset(res, 0, 257);
- a->deallocate(res);
-
- std::vector<void*> mems;
- for (int i = 0; i < 10000; i++) {
- mems.push_back(a->allocate(1));
- }
- for (auto& mem : mems) {
- a->deallocate(mem);
- }
- mems.clear();
-}
-
-#endif
std::is_standard_layout<CachelinePadded<int>>::value,
"CachelinePadded<T> must be standard-layout if T is.");
-const int kCachelineSize = folly::detail::CacheLocality::kFalseSharingRange;
+const int kCachelineSize = folly::CacheLocality::kFalseSharingRange;
template <int dataSize>
struct SizedData {
DeterministicSchedule::afterSharedAccess();
return rv;
}
+}
template <>
CacheLocality const& CacheLocality::system<test::DeterministicAtomic>() {
template <>
Getcpu::Func AccessSpreader<test::DeterministicAtomic>::pickGetcpuFunc() {
- return &DeterministicSchedule::getcpu;
-}
+ return &detail::DeterministicSchedule::getcpu;
}
}
#include <vector>
#include <folly/ScopeGuard.h>
+#include <folly/concurrency/CacheLocality.h>
#include <folly/detail/AtomicUtils.h>
-#include <folly/detail/CacheLocality.h>
#include <folly/detail/Futex.h>
#include <folly/portability/Semaphore.h>
std::chrono::time_point<std::chrono::system_clock>* absSystemTime,
std::chrono::time_point<std::chrono::steady_clock>* absSteadyTime,
uint32_t waitMask);
+}
template <>
Getcpu::Func AccessSpreader<test::DeterministicAtomic>::pickGetcpuFunc();
-}
+
} // namespace folly::detail