From 948400319a27dfcfc4cd5da408ba1a122a0b39da Mon Sep 17 00:00:00 2001 From: Daniel Byrne Date: Sun, 27 Feb 2022 22:57:34 -0500 Subject: [PATCH 01/13] initial implementation of background evictor thread --- cachelib/allocator/BackgroundEvictor-inl.h | 77 ++++++++++++++++ cachelib/allocator/BackgroundEvictor.h | 78 +++++++++++++++++ .../allocator/BackgroundEvictorStrategy.cpp | 42 +++++++++ .../allocator/BackgroundEvictorStrategy.h | 56 ++++++++++++ cachelib/allocator/CMakeLists.txt | 3 + cachelib/allocator/Cache.h | 6 ++ cachelib/allocator/CacheAllocator-inl.h | 29 +++++++ cachelib/allocator/CacheAllocator.h | 87 ++++++++++++++++++- cachelib/allocator/CacheAllocatorConfig.h | 27 ++++++ cachelib/allocator/CacheStats.h | 13 +++ cachelib/allocator/FreeThresholdStrategy.cpp | 63 ++++++++++++++ cachelib/allocator/FreeThresholdStrategy.h | 53 +++++++++++ cachelib/allocator/KeepFreeStrategy.cpp | 56 ++++++++++++ cachelib/allocator/KeepFreeStrategy.h | 48 ++++++++++ cachelib/cachebench/cache/Cache-inl.h | 4 + cachelib/cachebench/util/CacheConfig.cpp | 25 +++++- cachelib/cachebench/util/CacheConfig.h | 6 ++ 17 files changed, 671 insertions(+), 2 deletions(-) create mode 100644 cachelib/allocator/BackgroundEvictor-inl.h create mode 100644 cachelib/allocator/BackgroundEvictor.h create mode 100644 cachelib/allocator/BackgroundEvictorStrategy.cpp create mode 100644 cachelib/allocator/BackgroundEvictorStrategy.h create mode 100644 cachelib/allocator/FreeThresholdStrategy.cpp create mode 100644 cachelib/allocator/FreeThresholdStrategy.h create mode 100644 cachelib/allocator/KeepFreeStrategy.cpp create mode 100644 cachelib/allocator/KeepFreeStrategy.h diff --git a/cachelib/allocator/BackgroundEvictor-inl.h b/cachelib/allocator/BackgroundEvictor-inl.h new file mode 100644 index 0000000000..836defb671 --- /dev/null +++ b/cachelib/allocator/BackgroundEvictor-inl.h @@ -0,0 +1,77 @@ +/* + * Copyright (c) Intel and its affiliates. + * + * 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. + */ + + + +namespace facebook { +namespace cachelib { + + +template +BackgroundEvictor::BackgroundEvictor(Cache& cache, + std::shared_ptr strategy, + unsigned int tid) + : cache_(cache), + strategy_(strategy), + tid_(tid) { + +} + +template +BackgroundEvictor::~BackgroundEvictor() { stop(std::chrono::seconds(0)); } + +template +void BackgroundEvictor::work() { + try { + for (const auto pid : cache_.getRegularPoolIds()) { + //check if we exceed threshold (per class basis static right now) + checkAndRun(pid); + } + } catch (const std::exception& ex) { + XLOGF(ERR, "BackgroundEvictor interrupted due to exception: {}", ex.what()); + } +} + + +// Look for classes that exceed the target memory capacity +// and return those for eviction +template +void BackgroundEvictor::checkAndRun(PoolId pid) { + + const auto& mpStats = cache_.getPoolByTid(pid,tid_).getStats(); + for (auto& cid : mpStats.classIds) { + if (strategy_->shouldEvict(cache_,tid_,pid,cid)) { + unsigned int batch = strategy_->calculateBatchSize(cache_,tid_,pid,cid); + //try evicting BATCH items from the class in order to reach free target + unsigned int evicted = + BackgroundEvictorAPIWrapper::traverseAndEvictItems(cache_, + tid_,pid,cid,batch); + numEvictedItems_ += evicted; + } + } + runCount_ = runCount_ + 1; +} + +template +BackgroundEvictorStats BackgroundEvictor::getStats() const noexcept { + BackgroundEvictorStats stats; + stats.numEvictedItems = numEvictedItems_.load(std::memory_order_relaxed); + stats.numTraversals = runCount_.load(std::memory_order_relaxed); + return stats; +} + +} // namespace cachelib +} // namespace facebook diff --git a/cachelib/allocator/BackgroundEvictor.h b/cachelib/allocator/BackgroundEvictor.h new file mode 100644 index 0000000000..b53b0f7593 --- /dev/null +++ b/cachelib/allocator/BackgroundEvictor.h @@ -0,0 +1,78 @@ +/* + * Copyright (c) Intel and its affiliates. + * + * 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 + +#include "cachelib/allocator/CacheStats.h" +#include "cachelib/common/PeriodicWorker.h" +#include "cachelib/allocator/BackgroundEvictorStrategy.h" + + +namespace facebook { +namespace cachelib { + +// wrapper that exposes the private APIs of CacheType that are specifically +// needed for the eviction. +template +struct BackgroundEvictorAPIWrapper { + + static unsigned int traverseAndEvictItems(C& cache, + unsigned int tid, unsigned int pid, unsigned int cid, unsigned int batch) { + return cache.traverseAndEvictItems(tid,pid,cid,batch); + } +}; + +// Periodic worker that evicts items from tiers in batches +// The primary aim is to reduce insertion times for new items in the +// cache +template +class BackgroundEvictor : public PeriodicWorker { + public: + using Cache = CacheT; + // @param cache the cache interface + // @param target_free the target amount of memory to keep free in + // this tier + // @param tier id memory tier to perform eviction on + BackgroundEvictor(Cache& cache, + std::shared_ptr strategy, + unsigned int tid); + + ~BackgroundEvictor() override; + + BackgroundEvictorStats getStats() const noexcept; + + private: + // cache allocator's interface for evicting + + using Item = typename Cache::Item; + + Cache& cache_; + std::shared_ptr strategy_; + unsigned int tid_; + + // implements the actual logic of running the background evictor + void work() override final; + void checkAndRun(PoolId pid); + + std::atomic numEvictedItems_{0}; + std::atomic runCount_{0}; +}; +} // namespace cachelib +} // namespace facebook + +#include "cachelib/allocator/BackgroundEvictor-inl.h" diff --git a/cachelib/allocator/BackgroundEvictorStrategy.cpp b/cachelib/allocator/BackgroundEvictorStrategy.cpp new file mode 100644 index 0000000000..0cb7218c22 --- /dev/null +++ b/cachelib/allocator/BackgroundEvictorStrategy.cpp @@ -0,0 +1,42 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * 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 "cachelib/allocator/BackgroundEvictorStrategy.h" + +#include + +namespace facebook { +namespace cachelib { + + +bool BackgroundEvictorStrategy::shouldEvict(const CacheBase& cache, + unsigned int tid, + PoolId pid, + ClassId cid ) { + return true; +} + + +unsigned int BackgroundEvictorStrategy::calculateBatchSize(const CacheBase& cache, + unsigned int tid, + PoolId pid, + ClassId cid ) { + return 1; +} + + +} // namespace cachelib +} // namespace facebook diff --git a/cachelib/allocator/BackgroundEvictorStrategy.h b/cachelib/allocator/BackgroundEvictorStrategy.h new file mode 100644 index 0000000000..3b367f0184 --- /dev/null +++ b/cachelib/allocator/BackgroundEvictorStrategy.h @@ -0,0 +1,56 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * 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 "cachelib/allocator/Cache.h" + +namespace facebook { +namespace cachelib { + +//struct RebalanceContext { +// // Victim and Receiver must belong to the same pool +// ClassId victimClassId{Slab::kInvalidClassId}; +// ClassId receiverClassId{Slab::kInvalidClassId}; +// +// RebalanceContext() = default; +// RebalanceContext(ClassId victim, ClassId receiver) +// : victimClassId(victim), receiverClassId(receiver) {} +//}; + +// Base class for background eviction strategy. +class BackgroundEvictorStrategy { + +public: + BackgroundEvictorStrategy() {} + + ~BackgroundEvictorStrategy() {} + + unsigned int calculateBatchSize(const CacheBase& cache, + unsigned int tid, + PoolId pid, + ClassId cid ); + + bool shouldEvict(const CacheBase& cache, + unsigned int tid, + PoolId pid, + ClassId cid ); + + +}; + +} // namespace cachelib +} // namespace facebook diff --git a/cachelib/allocator/CMakeLists.txt b/cachelib/allocator/CMakeLists.txt index b64d48d86f..d8c1a367b0 100644 --- a/cachelib/allocator/CMakeLists.txt +++ b/cachelib/allocator/CMakeLists.txt @@ -27,6 +27,7 @@ add_library (cachelib_allocator ${SERIALIZE_THRIFT_FILES} ${DATASTRUCT_SERIALIZE_THRIFT_FILES} ${MEMORY_SERIALIZE_THRIFT_FILES} + BackgroundEvictorStrategy.cpp CacheAllocator.cpp Cache.cpp CacheDetails.cpp @@ -35,8 +36,10 @@ add_library (cachelib_allocator CCacheManager.cpp ContainerTypes.cpp FreeMemStrategy.cpp + FreeThresholdStrategy.cpp HitsPerSlabStrategy.cpp LruTailAgeStrategy.cpp + KeepFreeStrategy.cpp MarginalHitsOptimizeStrategy.cpp MarginalHitsStrategy.cpp memory/AllocationClass.cpp diff --git a/cachelib/allocator/Cache.h b/cachelib/allocator/Cache.h index c4a48506d3..41536c3f13 100644 --- a/cachelib/allocator/Cache.h +++ b/cachelib/allocator/Cache.h @@ -93,6 +93,12 @@ class CacheBase { // // @param poolId The pool id to query virtual const MemoryPool& getPool(PoolId poolId) const = 0; + + // Get the reference to a memory pool using a tier id, for stats purposes + // + // @param poolId The pool id to query + // @param tierId The tier of the pool id + virtual const MemoryPool& getPoolByTid(PoolId poolId, TierId tid) const = 0; // Get Pool specific stats (regular pools). This includes stats from the // Memory Pool and also the cache. diff --git a/cachelib/allocator/CacheAllocator-inl.h b/cachelib/allocator/CacheAllocator-inl.h index c8c11c77f5..0ac4057e8c 100644 --- a/cachelib/allocator/CacheAllocator-inl.h +++ b/cachelib/allocator/CacheAllocator-inl.h @@ -325,6 +325,18 @@ void CacheAllocator::initWorkers() { config_.poolOptimizeStrategy, config_.ccacheOptimizeStepSizePercent); } + if (config_.poolOptimizerEnabled()) { + startNewPoolOptimizer(config_.regularPoolOptimizeInterval, + config_.compactCacheOptimizeInterval, + config_.poolOptimizeStrategy, + config_.ccacheOptimizeStepSizePercent); + } + + if (config_.backgroundEvictorEnabled()) { + startNewBackgroundEvictor(config_.backgroundEvictorInterval, + config_.backgroundEvictorStrategy, + 0); //right now default to tier 0); + } } template @@ -3381,6 +3393,7 @@ bool CacheAllocator::stopWorkers(std::chrono::seconds timeout) { success &= stopPoolResizer(timeout); success &= stopMemMonitor(timeout); success &= stopReaper(timeout); + success &= stopBackgroundEvictor(timeout); return success; } @@ -3661,6 +3674,7 @@ GlobalCacheStats CacheAllocator::getGlobalCacheStats() const { ret.nvmCacheEnabled = nvmCache_ ? nvmCache_->isEnabled() : false; ret.nvmUpTime = currTime - getNVMCacheCreationTime(); ret.reaperStats = getReaperStats(); + ret.backgroundEvictorStats = getBackgroundEvictorStats(); ret.numActiveHandles = getNumActiveHandles(); return ret; @@ -3759,6 +3773,7 @@ bool CacheAllocator::startNewPoolRebalancer( freeAllocThreshold); } + template bool CacheAllocator::startNewPoolResizer( std::chrono::milliseconds interval, @@ -3796,6 +3811,14 @@ bool CacheAllocator::startNewReaper( return startNewWorker("Reaper", reaper_, interval, reaperThrottleConfig); } +template +bool CacheAllocator::startNewBackgroundEvictor( + std::chrono::milliseconds interval, + std::shared_ptr strategy, + unsigned int tid ) { + return startNewWorker("BackgroundEvictor", backgroundEvictor_, interval, strategy, tid); +} + template bool CacheAllocator::stopPoolRebalancer( std::chrono::seconds timeout) { @@ -3823,6 +3846,12 @@ bool CacheAllocator::stopReaper(std::chrono::seconds timeout) { return stopWorker("Reaper", reaper_, timeout); } +template +bool CacheAllocator::stopBackgroundEvictor( + std::chrono::seconds timeout) { + return stopWorker("BackgroundEvictor", backgroundEvictor_, timeout); +} + template bool CacheAllocator::cleanupStrayShmSegments( const std::string& cacheDir, bool posix /*TODO(SHM_FILE): const std::vector& config */) { diff --git a/cachelib/allocator/CacheAllocator.h b/cachelib/allocator/CacheAllocator.h index 319e66a626..af9b8b0644 100644 --- a/cachelib/allocator/CacheAllocator.h +++ b/cachelib/allocator/CacheAllocator.h @@ -36,7 +36,7 @@ #include #include #pragma GCC diagnostic pop - +#include "cachelib/allocator/BackgroundEvictor.h" #include "cachelib/allocator/CCacheManager.h" #include "cachelib/allocator/Cache.h" #include "cachelib/allocator/CacheAllocatorConfig.h" @@ -945,6 +945,9 @@ class CacheAllocator : public CacheBase { // @param reaperThrottleConfig throttling config bool startNewReaper(std::chrono::milliseconds interval, util::Throttler::Config reaperThrottleConfig); + + bool startNewBackgroundEvictor(std::chrono::milliseconds interval, + std::shared_ptr strategy, unsigned int tid); // Stop existing workers with a timeout bool stopPoolRebalancer(std::chrono::seconds timeout = std::chrono::seconds{ @@ -954,6 +957,7 @@ class CacheAllocator : public CacheBase { 0}); bool stopMemMonitor(std::chrono::seconds timeout = std::chrono::seconds{0}); bool stopReaper(std::chrono::seconds timeout = std::chrono::seconds{0}); + bool stopBackgroundEvictor(std::chrono::seconds timeout = std::chrono::seconds{0}); // Set pool optimization to either true or false // @@ -988,6 +992,10 @@ class CacheAllocator : public CacheBase { const MemoryPool& getPool(PoolId pid) const override final { return allocator_[currentTier()]->getPool(pid); } + + const MemoryPool& getPoolByTid(PoolId pid, TierId tid) const override final { + return allocator_[tid]->getPool(pid); + } // calculate the number of slabs to be advised/reclaimed in each pool PoolAdviseReclaimData calcNumSlabsToAdviseReclaim() override final { @@ -1034,6 +1042,12 @@ class CacheAllocator : public CacheBase { auto stats = reaper_ ? reaper_->getStats() : ReaperStats{}; return stats; } + + // returns the background evictor + BackgroundEvictorStats getBackgroundEvictorStats() const { + auto stats = backgroundEvictor_ ? backgroundEvictor_->getStats() : BackgroundEvictorStats{}; + return stats; + } // return the LruType of an item typename MMType::LruType getItemLruType(const Item& item) const; @@ -1752,6 +1766,73 @@ class CacheAllocator : public CacheBase { folly::annotate_ignore_thread_sanitizer_guard g(__FILE__, __LINE__); allocator_[currentTier()]->forEachAllocation(std::forward(f)); } + + // exposed for the background evictor to iterate through the memory and evict + // in batch. This should improve insertion path for tiered memory config + unsigned int traverseAndEvictItems(unsigned int tid, unsigned int pid, unsigned int cid, unsigned int batch) { + + auto& mmContainer = getMMContainer(tid, pid, cid); + unsigned int evictions = 0; + auto itr = mmContainer.getEvictionIterator(); + while (evictions < batch && itr) { + + Item* candidate = itr.get(); + // for chained items, the ownership of the parent can change. We try to + // evict what we think as parent and see if the eviction of parent + // recycles the child we intend to. + + ItemHandle toReleaseHandle = tryEvictToNextMemoryTier(tid, pid, itr); + bool movedToNextTier = false; + if(toReleaseHandle) { + movedToNextTier = true; + } else { + toReleaseHandle = + itr->isChainedItem() + ? advanceIteratorAndTryEvictChainedItem(tid, pid, itr) + : advanceIteratorAndTryEvictRegularItem(tid, pid, mmContainer, itr); + } + + if (toReleaseHandle) { + if (toReleaseHandle->hasChainedItem()) { + (*stats_.chainedItemEvictions)[pid][cid].inc(); + } else { + (*stats_.regularItemEvictions)[pid][cid].inc(); + } + ++evictions; + + + // we must be the last handle and for chained items, this will be + // the parent. + XDCHECK(toReleaseHandle.get() == candidate || candidate->isChainedItem()); + XDCHECK_EQ(1u, toReleaseHandle->getRefCount()); + + // We manually release the item here because we don't want to + // invoke the Item Handle's destructor which will be decrementing + // an already zero refcount, which will throw exception + auto& itemToRelease = *toReleaseHandle.release(); + + // Decrementing the refcount because we want to recycle the item + const auto ref = decRef(itemToRelease); + XDCHECK_EQ(0u, ref); + + // check if by releasing the item we intend to, we actually + // recycle the candidate. + releaseBackToAllocator(itemToRelease, RemoveContext::kEviction, + /* isNascent */ movedToNextTier, candidate); + + } + + // If we destroyed the itr to possibly evict and failed, we restart + // from the beginning again + if (!itr) { + itr.resetToBegin(); + } + } + // Invalidate iterator since later on we may use this mmContainer + // again, which cannot be done unless we drop this iterator + itr.destroy(); + return evictions; + } // returns true if nvmcache is enabled and we should write this item to // nvmcache. @@ -2064,6 +2145,9 @@ class CacheAllocator : public CacheBase { // free memory monitor std::unique_ptr memMonitor_; + + // background evictor + std::unique_ptr> backgroundEvictor_; // check whether a pool is a slabs pool std::array isCompactCachePool_{}; @@ -2119,6 +2203,7 @@ class CacheAllocator : public CacheBase { // Make this friend to give access to acquire and release friend ReadHandle; friend ReaperAPIWrapper; + friend BackgroundEvictorAPIWrapper; friend class CacheAPIWrapperForNvm; friend class FbInternalRuntimeUpdateWrapper; diff --git a/cachelib/allocator/CacheAllocatorConfig.h b/cachelib/allocator/CacheAllocatorConfig.h index 1d11b3ef14..118e8437b5 100644 --- a/cachelib/allocator/CacheAllocatorConfig.h +++ b/cachelib/allocator/CacheAllocatorConfig.h @@ -265,6 +265,12 @@ class CacheAllocatorConfig { std::chrono::seconds regularInterval, std::chrono::seconds ccacheInterval, uint32_t ccacheStepSizePercent); + + // Enable the background evictor - scans a tier to look for objects + // to evict to the next tier + CacheAllocatorConfig& enableBackgroundEvictor( + std::shared_ptr backgroundEvictorStrategy, + std::chrono::seconds regularInterval); // This enables an optimization for Pool rebalancing and resizing. // The rough idea is to ensure only the least useful items are evicted when @@ -336,6 +342,12 @@ class CacheAllocatorConfig { compactCacheOptimizeInterval.count() > 0) && poolOptimizeStrategy != nullptr; } + + // @return whether background evictor thread is enabled + bool backgroundEvictorEnabled() const noexcept { + return backgroundEvictorInterval.count() > 0 && + backgroundEvictorStrategy != nullptr; + } // @return whether memory monitor is enabled bool memMonitoringEnabled() const noexcept { @@ -426,6 +438,9 @@ class CacheAllocatorConfig { // time interval to sleep between iterators of rebalancing the pools. std::chrono::milliseconds poolRebalanceInterval{std::chrono::seconds{1}}; + + // time interval to sleep between runs of the background evictor + std::chrono::milliseconds backgroundEvictorInterval{std::chrono::seconds{1}}; // Free slabs pro-actively if the ratio of number of freeallocs to // the number of allocs per slab in a slab class is above this @@ -437,6 +452,9 @@ class CacheAllocatorConfig { // rebalance to avoid alloc fialures. std::shared_ptr defaultPoolRebalanceStrategy{ new RebalanceStrategy{}}; + + // rebalance to avoid alloc fialures. + std::shared_ptr backgroundEvictorStrategy; // time interval to sleep between iterations of pool size optimization, // for regular pools and compact caches @@ -963,6 +981,15 @@ CacheAllocatorConfig& CacheAllocatorConfig::enablePoolRebalancing( return *this; } +template +CacheAllocatorConfig& CacheAllocatorConfig::enableBackgroundEvictor( + std::shared_ptr strategy, + std::chrono::seconds interval) { + backgroundEvictorStrategy = strategy; + backgroundEvictorInterval = interval; + return *this; +} + template CacheAllocatorConfig& CacheAllocatorConfig::enablePoolResizing( std::shared_ptr resizeStrategy, diff --git a/cachelib/allocator/CacheStats.h b/cachelib/allocator/CacheStats.h index 146de6bea7..2da0e82d57 100644 --- a/cachelib/allocator/CacheStats.h +++ b/cachelib/allocator/CacheStats.h @@ -285,6 +285,16 @@ struct ReaperStats { uint64_t avgTraversalTimeMs{0}; }; +// Stats for background evictor +struct BackgroundEvictorStats { + // the total number of items the reaper has visited. + uint64_t numEvictedItems{0}; + + // number of times we went executed the thread //TODO: is this def correct? + uint64_t numTraversals{0}; + +}; + // CacheMetadata type to export struct CacheMetadata { // allocator_version @@ -469,6 +479,9 @@ struct GlobalCacheStats { // stats related to the reaper ReaperStats reaperStats; + + // stats related to the background evictor + BackgroundEvictorStats backgroundEvictorStats; uint64_t numNvmRejectsByExpiry{}; uint64_t numNvmRejectsByClean{}; diff --git a/cachelib/allocator/FreeThresholdStrategy.cpp b/cachelib/allocator/FreeThresholdStrategy.cpp new file mode 100644 index 0000000000..efbbfbf1ee --- /dev/null +++ b/cachelib/allocator/FreeThresholdStrategy.cpp @@ -0,0 +1,63 @@ +/* + * Copyright (c) Intel and its affiliates. + * + * 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 "cachelib/allocator/FreeThresholdStrategy.h" + +#include + +namespace facebook { +namespace cachelib { + + + +FreeThresholdStrategy::FreeThresholdStrategy(double freeThreshold) + : freeThreshold_(freeThreshold) {} + +bool FreeThresholdStrategy::shouldEvict(const CacheBase& cache, + unsigned int tid, + PoolId pid, + ClassId cid ) { + + const auto& mpStats = cache.getPoolByTid(pid,tid).getStats(); + size_t totalMem = mpStats.acStats.at(cid).getTotalMemory(); + size_t freeMem = mpStats.acStats.at(cid).getTotalFreeMemory(); + + double currFree = (double)freeMem/(double)totalMem; + if (currFree < freeThreshold_) { + return true; + } else { + return false; + } + +} + +unsigned int FreeThresholdStrategy::calculateBatchSize(const CacheBase& cache, + unsigned int tid, + PoolId pid, + ClassId cid ) { + const auto& mpStats = cache.getPoolByTid(pid,tid).getStats(); + size_t totalMem = mpStats.acStats.at(cid).getTotalMemory(); + size_t freeMem = mpStats.acStats.at(cid).getTotalFreeMemory(); + + size_t targetMem = (freeThreshold_ * totalMem) - freeMem; + unsigned int batch = (targetMem / mpStats.acStats.at(cid).allocSize); + + return batch; +} + + +} // namespace cachelib +} // namespace facebook diff --git a/cachelib/allocator/FreeThresholdStrategy.h b/cachelib/allocator/FreeThresholdStrategy.h new file mode 100644 index 0000000000..e0954e888d --- /dev/null +++ b/cachelib/allocator/FreeThresholdStrategy.h @@ -0,0 +1,53 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * 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 "cachelib/allocator/Cache.h" +#include "cachelib/allocator/BackgroundEvictorStrategy.h" + +namespace facebook { +namespace cachelib { + + +// Base class for background eviction strategy. +class FreeThresholdStrategy : public BackgroundEvictorStrategy { + +public: + + FreeThresholdStrategy(double freeThreshold); + + ~FreeThresholdStrategy() {} + + unsigned int calculateBatchSize(const CacheBase& cache, + unsigned int tid, + PoolId pid, + ClassId cid ); + + bool shouldEvict(const CacheBase& cache, + unsigned int tid, + PoolId pid, + ClassId cid ); + +private: + + double freeThreshold_; + + +}; + +} // namespace cachelib +} // namespace facebook diff --git a/cachelib/allocator/KeepFreeStrategy.cpp b/cachelib/allocator/KeepFreeStrategy.cpp new file mode 100644 index 0000000000..b894ab1f92 --- /dev/null +++ b/cachelib/allocator/KeepFreeStrategy.cpp @@ -0,0 +1,56 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * 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 "cachelib/allocator/KeepFreeStrategy.h" + +#include + +namespace facebook { +namespace cachelib { + + +KeepFreeStrategy::KeepFreeStrategy(unsigned int nKeepFree) + : nKeepFree_(nKeepFree) {} + +bool KeepFreeStrategy::shouldEvict(const CacheBase& cache, + unsigned int tid, + PoolId pid, + ClassId cid ) { + const auto& mpStats = cache.getPoolByTid(pid,tid).getStats(); + size_t allocSize = mpStats.acStats.at(cid).allocSize; + size_t freeAllocs = mpStats.acStats.at(cid).getTotalFreeMemory() / allocSize; + if (freeAllocs < nKeepFree_) { + return true; + } else { + return false; + } + +} + +unsigned int KeepFreeStrategy::calculateBatchSize(const CacheBase& cache, + unsigned int tid, + PoolId pid, + ClassId cid ) { + const auto& mpStats = cache.getPoolByTid(pid,tid).getStats(); + size_t allocSize = mpStats.acStats.at(cid).allocSize; + size_t freeAllocs = mpStats.acStats.at(cid).getTotalFreeMemory() / allocSize; + + return (unsigned int)(nKeepFree_ - freeAllocs); +} + + +} // namespace cachelib +} // namespace facebook diff --git a/cachelib/allocator/KeepFreeStrategy.h b/cachelib/allocator/KeepFreeStrategy.h new file mode 100644 index 0000000000..70fcabbeab --- /dev/null +++ b/cachelib/allocator/KeepFreeStrategy.h @@ -0,0 +1,48 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * 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 "cachelib/allocator/Cache.h" +#include "cachelib/allocator/BackgroundEvictorStrategy.h" + +namespace facebook { +namespace cachelib { + +class KeepFreeStrategy : public BackgroundEvictorStrategy { + +public: + + KeepFreeStrategy(unsigned int nKeepFree); + + ~KeepFreeStrategy() {} + + unsigned int calculateBatchSize(const CacheBase& cache, + unsigned int tid, + PoolId pid, + ClassId cid ); + + bool shouldEvict(const CacheBase& cache, + unsigned int tid, + PoolId pid, + ClassId cid ); +private: + unsigned int nKeepFree_; + +}; + +} // namespace cachelib +} // namespace facebook diff --git a/cachelib/cachebench/cache/Cache-inl.h b/cachelib/cachebench/cache/Cache-inl.h index 34f65e1b15..174f94b15d 100644 --- a/cachelib/cachebench/cache/Cache-inl.h +++ b/cachelib/cachebench/cache/Cache-inl.h @@ -59,6 +59,10 @@ Cache::Cache(const CacheConfig& config, allocatorConfig_.enablePoolRebalancing( config_.getRebalanceStrategy(), std::chrono::seconds(config_.poolRebalanceIntervalSec)); + + allocatorConfig_.enableBackgroundEvictor( + config_.getBackgroundEvictorStrategy(), + std::chrono::seconds(config_.backgroundEvictorIntervalSec)); if (config_.moveOnSlabRelease && movingSync != nullptr) { allocatorConfig_.enableMovingOnSlabRelease( diff --git a/cachelib/cachebench/util/CacheConfig.cpp b/cachelib/cachebench/util/CacheConfig.cpp index 2604744bd9..26cc2eef14 100644 --- a/cachelib/cachebench/util/CacheConfig.cpp +++ b/cachelib/cachebench/util/CacheConfig.cpp @@ -20,6 +20,9 @@ #include "cachelib/allocator/LruTailAgeStrategy.h" #include "cachelib/allocator/RandomStrategy.h" +#include "cachelib/allocator/KeepFreeStrategy.h" +#include "cachelib/allocator/FreeThresholdStrategy.h" + namespace facebook { namespace cachelib { namespace cachebench { @@ -27,10 +30,15 @@ CacheConfig::CacheConfig(const folly::dynamic& configJson) { JSONSetVal(configJson, allocator); JSONSetVal(configJson, cacheSizeMB); JSONSetVal(configJson, poolRebalanceIntervalSec); + JSONSetVal(configJson, backgroundEvictorIntervalSec); JSONSetVal(configJson, moveOnSlabRelease); JSONSetVal(configJson, rebalanceStrategy); JSONSetVal(configJson, rebalanceMinSlabs); JSONSetVal(configJson, rebalanceDiffRatio); + + JSONSetVal(configJson, backgroundEvictorStrategy); + JSONSetVal(configJson, freeThreshold); + JSONSetVal(configJson, nKeepFree); JSONSetVal(configJson, htBucketPower); JSONSetVal(configJson, htLockPower); @@ -104,7 +112,7 @@ CacheConfig::CacheConfig(const folly::dynamic& configJson) { // if you added new fields to the configuration, update the JSONSetVal // to make them available for the json configs and increment the size // below - checkCorrectSize(); + checkCorrectSize(); if (numPools != poolSizes.size()) { throw std::invalid_argument(folly::sformat( @@ -134,6 +142,21 @@ std::shared_ptr CacheConfig::getRebalanceStrategy() const { } } +std::shared_ptr CacheConfig::getBackgroundEvictorStrategy() const { + if (backgroundEvictorIntervalSec == 0) { + return nullptr; + } + + if (backgroundEvictorStrategy == "free-threshold") { + return std::make_shared(freeThreshold); + } else if (backgroundEvictorStrategy == "keep-free") { + return std::make_shared(nKeepFree); + } else { + //default! + return std::make_shared(freeThreshold); + } +} + MemoryTierConfig::MemoryTierConfig(const folly::dynamic& configJson) { JSONSetVal(configJson, file); diff --git a/cachelib/cachebench/util/CacheConfig.h b/cachelib/cachebench/util/CacheConfig.h index f09d5966bd..7c464672d6 100644 --- a/cachelib/cachebench/util/CacheConfig.h +++ b/cachelib/cachebench/util/CacheConfig.h @@ -20,6 +20,7 @@ #include "cachelib/allocator/CacheAllocator.h" #include "cachelib/allocator/RebalanceStrategy.h" +#include "cachelib/allocator/BackgroundEvictorStrategy.h" #include "cachelib/cachebench/util/JSONConfig.h" #include "cachelib/common/Ticker.h" #include "cachelib/navy/common/Device.h" @@ -71,7 +72,11 @@ struct CacheConfig : public JSONConfig { uint64_t cacheSizeMB{0}; uint64_t poolRebalanceIntervalSec{0}; + uint64_t backgroundEvictorIntervalSec{0}; std::string rebalanceStrategy; + std::string backgroundEvictorStrategy; + double freeThreshold{0.01}; //keep 1% of space free + uint32_t nKeepFree{100}; //keep at most 100 slots free uint64_t rebalanceMinSlabs{1}; double rebalanceDiffRatio{0.25}; bool moveOnSlabRelease{false}; @@ -288,6 +293,7 @@ struct CacheConfig : public JSONConfig { CacheConfig() {} std::shared_ptr getRebalanceStrategy() const; + std::shared_ptr getBackgroundEvictorStrategy() const; }; } // namespace cachebench } // namespace cachelib From eea8127d801a870748262ebc929983691d1ac0c8 Mon Sep 17 00:00:00 2001 From: Daniel Byrne Date: Mon, 28 Feb 2022 10:11:55 -0500 Subject: [PATCH 02/13] bug fixes - changed timer to MS for bg evict --- .../allocator/BackgroundEvictorStrategy.cpp | 42 ------------------- .../allocator/BackgroundEvictorStrategy.h | 11 ++--- cachelib/allocator/CMakeLists.txt | 1 - cachelib/allocator/CacheAllocatorConfig.h | 6 +-- cachelib/cachebench/cache/Cache-inl.h | 5 ++- cachelib/cachebench/cache/CacheStats.h | 8 ++++ cachelib/cachebench/util/CacheConfig.cpp | 4 +- cachelib/cachebench/util/CacheConfig.h | 2 +- 8 files changed, 22 insertions(+), 57 deletions(-) delete mode 100644 cachelib/allocator/BackgroundEvictorStrategy.cpp diff --git a/cachelib/allocator/BackgroundEvictorStrategy.cpp b/cachelib/allocator/BackgroundEvictorStrategy.cpp deleted file mode 100644 index 0cb7218c22..0000000000 --- a/cachelib/allocator/BackgroundEvictorStrategy.cpp +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * 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 "cachelib/allocator/BackgroundEvictorStrategy.h" - -#include - -namespace facebook { -namespace cachelib { - - -bool BackgroundEvictorStrategy::shouldEvict(const CacheBase& cache, - unsigned int tid, - PoolId pid, - ClassId cid ) { - return true; -} - - -unsigned int BackgroundEvictorStrategy::calculateBatchSize(const CacheBase& cache, - unsigned int tid, - PoolId pid, - ClassId cid ) { - return 1; -} - - -} // namespace cachelib -} // namespace facebook diff --git a/cachelib/allocator/BackgroundEvictorStrategy.h b/cachelib/allocator/BackgroundEvictorStrategy.h index 3b367f0184..ae053ee9b5 100644 --- a/cachelib/allocator/BackgroundEvictorStrategy.h +++ b/cachelib/allocator/BackgroundEvictorStrategy.h @@ -35,19 +35,16 @@ namespace cachelib { class BackgroundEvictorStrategy { public: - BackgroundEvictorStrategy() {} - ~BackgroundEvictorStrategy() {} - - unsigned int calculateBatchSize(const CacheBase& cache, + virtual unsigned int calculateBatchSize(const CacheBase& cache, unsigned int tid, PoolId pid, - ClassId cid ); + ClassId cid ) = 0; - bool shouldEvict(const CacheBase& cache, + virtual bool shouldEvict(const CacheBase& cache, unsigned int tid, PoolId pid, - ClassId cid ); + ClassId cid ) = 0; }; diff --git a/cachelib/allocator/CMakeLists.txt b/cachelib/allocator/CMakeLists.txt index d8c1a367b0..b250923ba7 100644 --- a/cachelib/allocator/CMakeLists.txt +++ b/cachelib/allocator/CMakeLists.txt @@ -27,7 +27,6 @@ add_library (cachelib_allocator ${SERIALIZE_THRIFT_FILES} ${DATASTRUCT_SERIALIZE_THRIFT_FILES} ${MEMORY_SERIALIZE_THRIFT_FILES} - BackgroundEvictorStrategy.cpp CacheAllocator.cpp Cache.cpp CacheDetails.cpp diff --git a/cachelib/allocator/CacheAllocatorConfig.h b/cachelib/allocator/CacheAllocatorConfig.h index 118e8437b5..c4a7faad06 100644 --- a/cachelib/allocator/CacheAllocatorConfig.h +++ b/cachelib/allocator/CacheAllocatorConfig.h @@ -270,7 +270,7 @@ class CacheAllocatorConfig { // to evict to the next tier CacheAllocatorConfig& enableBackgroundEvictor( std::shared_ptr backgroundEvictorStrategy, - std::chrono::seconds regularInterval); + std::chrono::milliseconds regularInterval); // This enables an optimization for Pool rebalancing and resizing. // The rough idea is to ensure only the least useful items are evicted when @@ -440,7 +440,7 @@ class CacheAllocatorConfig { std::chrono::milliseconds poolRebalanceInterval{std::chrono::seconds{1}}; // time interval to sleep between runs of the background evictor - std::chrono::milliseconds backgroundEvictorInterval{std::chrono::seconds{1}}; + std::chrono::milliseconds backgroundEvictorInterval{std::chrono::milliseconds{1000}}; // Free slabs pro-actively if the ratio of number of freeallocs to // the number of allocs per slab in a slab class is above this @@ -984,7 +984,7 @@ CacheAllocatorConfig& CacheAllocatorConfig::enablePoolRebalancing( template CacheAllocatorConfig& CacheAllocatorConfig::enableBackgroundEvictor( std::shared_ptr strategy, - std::chrono::seconds interval) { + std::chrono::milliseconds interval) { backgroundEvictorStrategy = strategy; backgroundEvictorInterval = interval; return *this; diff --git a/cachelib/cachebench/cache/Cache-inl.h b/cachelib/cachebench/cache/Cache-inl.h index 174f94b15d..2ada844d3d 100644 --- a/cachelib/cachebench/cache/Cache-inl.h +++ b/cachelib/cachebench/cache/Cache-inl.h @@ -62,7 +62,7 @@ Cache::Cache(const CacheConfig& config, allocatorConfig_.enableBackgroundEvictor( config_.getBackgroundEvictorStrategy(), - std::chrono::seconds(config_.backgroundEvictorIntervalSec)); + std::chrono::milliseconds(config_.backgroundEvictorIntervalMilSec)); if (config_.moveOnSlabRelease && movingSync != nullptr) { allocatorConfig_.enableMovingOnSlabRelease( @@ -523,6 +523,9 @@ Stats Cache::getStats() const { ret.numItems = aggregate.numItems(); ret.allocAttempts = cacheStats.allocAttempts; ret.allocFailures = cacheStats.allocFailures; + + ret.numBackgroundEvictions = cacheStats.backgroundEvictorStats.numEvictedItems; + ret.numBackgroundEvictorRuns = cacheStats.backgroundEvictorStats.numTraversals; ret.numCacheGets = cacheStats.numCacheGets; ret.numCacheGetMiss = cacheStats.numCacheGetMiss; diff --git a/cachelib/cachebench/cache/CacheStats.h b/cachelib/cachebench/cache/CacheStats.h index 004f9fe4c7..6285fcb3c2 100644 --- a/cachelib/cachebench/cache/CacheStats.h +++ b/cachelib/cachebench/cache/CacheStats.h @@ -32,6 +32,9 @@ struct Stats { uint64_t allocAttempts{0}; uint64_t allocFailures{0}; + uint64_t numBackgroundEvictions{0}; + uint64_t numBackgroundEvictorRuns{0}; + uint64_t numCacheGets{0}; uint64_t numCacheGetMiss{0}; uint64_t numRamDestructorCalls{0}; @@ -114,6 +117,11 @@ struct Stats { invertPctFn(allocFailures, allocAttempts)) << std::endl; out << folly::sformat("RAM Evictions : {:,}", numEvictions) << std::endl; + + + out << folly::sformat("Background Tier 0 Evictions : {:,}", numBackgroundEvictions) << std::endl; + + out << folly::sformat("Background Tier 0 Eviction Runs : {:,}", numBackgroundEvictorRuns) << std::endl; if (numCacheGets > 0) { out << folly::sformat("Cache Gets : {:,}", numCacheGets) << std::endl; diff --git a/cachelib/cachebench/util/CacheConfig.cpp b/cachelib/cachebench/util/CacheConfig.cpp index 26cc2eef14..bd0ae5d3b8 100644 --- a/cachelib/cachebench/util/CacheConfig.cpp +++ b/cachelib/cachebench/util/CacheConfig.cpp @@ -30,7 +30,7 @@ CacheConfig::CacheConfig(const folly::dynamic& configJson) { JSONSetVal(configJson, allocator); JSONSetVal(configJson, cacheSizeMB); JSONSetVal(configJson, poolRebalanceIntervalSec); - JSONSetVal(configJson, backgroundEvictorIntervalSec); + JSONSetVal(configJson, backgroundEvictorIntervalMilSec); JSONSetVal(configJson, moveOnSlabRelease); JSONSetVal(configJson, rebalanceStrategy); JSONSetVal(configJson, rebalanceMinSlabs); @@ -143,7 +143,7 @@ std::shared_ptr CacheConfig::getRebalanceStrategy() const { } std::shared_ptr CacheConfig::getBackgroundEvictorStrategy() const { - if (backgroundEvictorIntervalSec == 0) { + if (backgroundEvictorIntervalMilSec == 0) { return nullptr; } diff --git a/cachelib/cachebench/util/CacheConfig.h b/cachelib/cachebench/util/CacheConfig.h index 7c464672d6..e51470555f 100644 --- a/cachelib/cachebench/util/CacheConfig.h +++ b/cachelib/cachebench/util/CacheConfig.h @@ -72,7 +72,7 @@ struct CacheConfig : public JSONConfig { uint64_t cacheSizeMB{0}; uint64_t poolRebalanceIntervalSec{0}; - uint64_t backgroundEvictorIntervalSec{0}; + uint64_t backgroundEvictorIntervalMilSec{0}; std::string rebalanceStrategy; std::string backgroundEvictorStrategy; double freeThreshold{0.01}; //keep 1% of space free From 6bfed82eebcfd8d0862f255eb9fbfe0d6a189900 Mon Sep 17 00:00:00 2001 From: Daniel Byrne Date: Tue, 1 Mar 2022 12:01:20 -0500 Subject: [PATCH 03/13] fixed bug where bg thread gets stuck on failure + compile error --- cachelib/allocator/CacheAllocator.h | 12 +++++++----- cachelib/allocator/CacheAllocatorConfig.h | 1 + 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/cachelib/allocator/CacheAllocator.h b/cachelib/allocator/CacheAllocator.h index af9b8b0644..3a2aae1a94 100644 --- a/cachelib/allocator/CacheAllocator.h +++ b/cachelib/allocator/CacheAllocator.h @@ -1777,6 +1777,9 @@ class CacheAllocator : public CacheBase { while (evictions < batch && itr) { Item* candidate = itr.get(); + if (candidate == NULL) { + break; + } // for chained items, the ownership of the parent can change. We try to // evict what we think as parent and see if the eviction of parent // recycles the child we intend to. @@ -1820,14 +1823,13 @@ class CacheAllocator : public CacheBase { releaseBackToAllocator(itemToRelease, RemoveContext::kEviction, /* isNascent */ movedToNextTier, candidate); + } else { + //evict failed - let's quit with what we have + break; } - // If we destroyed the itr to possibly evict and failed, we restart - // from the beginning again - if (!itr) { - itr.resetToBegin(); - } } + // Invalidate iterator since later on we may use this mmContainer // again, which cannot be done unless we drop this iterator itr.destroy(); diff --git a/cachelib/allocator/CacheAllocatorConfig.h b/cachelib/allocator/CacheAllocatorConfig.h index c4a7faad06..97eec11411 100644 --- a/cachelib/allocator/CacheAllocatorConfig.h +++ b/cachelib/allocator/CacheAllocatorConfig.h @@ -31,6 +31,7 @@ #include "cachelib/allocator/NvmAdmissionPolicy.h" #include "cachelib/allocator/PoolOptimizeStrategy.h" #include "cachelib/allocator/RebalanceStrategy.h" +#include "cachelib/allocator/BackgroundEvictorStrategy.h" #include "cachelib/allocator/Util.h" #include "cachelib/common/EventInterface.h" #include "cachelib/common/Throttler.h" From d43e0e78ed3b8a4088f2200115f1dfd4414a838d Mon Sep 17 00:00:00 2001 From: Daniel Byrne Date: Thu, 3 Mar 2022 10:45:44 -0500 Subject: [PATCH 04/13] just skip the bad objects --- cachelib/allocator/CacheAllocator.h | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/cachelib/allocator/CacheAllocator.h b/cachelib/allocator/CacheAllocator.h index 3a2aae1a94..ce22230e0a 100644 --- a/cachelib/allocator/CacheAllocator.h +++ b/cachelib/allocator/CacheAllocator.h @@ -1789,10 +1789,12 @@ class CacheAllocator : public CacheBase { if(toReleaseHandle) { movedToNextTier = true; } else { - toReleaseHandle = - itr->isChainedItem() - ? advanceIteratorAndTryEvictChainedItem(tid, pid, itr) - : advanceIteratorAndTryEvictRegularItem(tid, pid, mmContainer, itr); + //toReleaseHandle = + // itr->isChainedItem() + // ? advanceIteratorAndTryEvictChainedItem(tid, pid, itr) + // : advanceIteratorAndTryEvictRegularItem(tid, pid, mmContainer, itr); + //just bump the iterator and move on with life + ++itr; } if (toReleaseHandle) { @@ -1823,10 +1825,7 @@ class CacheAllocator : public CacheBase { releaseBackToAllocator(itemToRelease, RemoveContext::kEviction, /* isNascent */ movedToNextTier, candidate); - } else { - //evict failed - let's quit with what we have - break; - } + } } From d6f03e9da877f046f89fd99a2ba2e73c9cb64a20 Mon Sep 17 00:00:00 2001 From: Daniel Byrne Date: Thu, 3 Mar 2022 17:00:38 -0500 Subject: [PATCH 05/13] fixed build failure --- cachelib/allocator/memory/MemoryAllocatorStats.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/cachelib/allocator/memory/MemoryAllocatorStats.h b/cachelib/allocator/memory/MemoryAllocatorStats.h index 947deec664..8d74dc8fe1 100644 --- a/cachelib/allocator/memory/MemoryAllocatorStats.h +++ b/cachelib/allocator/memory/MemoryAllocatorStats.h @@ -54,6 +54,10 @@ struct ACStats { constexpr size_t getTotalFreeMemory() const noexcept { return Slab::kSize * freeSlabs + freeAllocs * allocSize; } + + constexpr size_t getTotalMemory() const noexcept { + return activeAllocs * allocSize; + } }; // structure to query stats corresponding to a MemoryPool From 26c8eb5ee5d7838c4ad783dc125e16c0d73ee836 Mon Sep 17 00:00:00 2001 From: Daniel Byrne Date: Mon, 14 Mar 2022 09:44:39 -0400 Subject: [PATCH 06/13] fixed issue with item's not being released back to allocator --- cachelib/allocator/BackgroundEvictor-inl.h | 6 ++++-- cachelib/allocator/CacheAllocator.h | 5 +++-- cachelib/allocator/FreeThresholdStrategy.cpp | 17 +++++++++++------ cachelib/allocator/KeepFreeStrategy.cpp | 18 ++++++++++-------- 4 files changed, 28 insertions(+), 18 deletions(-) diff --git a/cachelib/allocator/BackgroundEvictor-inl.h b/cachelib/allocator/BackgroundEvictor-inl.h index 836defb671..2cc8538e59 100644 --- a/cachelib/allocator/BackgroundEvictor-inl.h +++ b/cachelib/allocator/BackgroundEvictor-inl.h @@ -37,8 +37,10 @@ template void BackgroundEvictor::work() { try { for (const auto pid : cache_.getRegularPoolIds()) { - //check if we exceed threshold (per class basis static right now) - checkAndRun(pid); + //check if the pool is full - probably should be if tier is full + if (cache_.getPoolByTid(pid,tid_).allSlabsAllocated()) { + checkAndRun(pid); + } } } catch (const std::exception& ex) { XLOGF(ERR, "BackgroundEvictor interrupted due to exception: {}", ex.what()); diff --git a/cachelib/allocator/CacheAllocator.h b/cachelib/allocator/CacheAllocator.h index ce22230e0a..af6069e3a5 100644 --- a/cachelib/allocator/CacheAllocator.h +++ b/cachelib/allocator/CacheAllocator.h @@ -1822,8 +1822,9 @@ class CacheAllocator : public CacheBase { // check if by releasing the item we intend to, we actually // recycle the candidate. - releaseBackToAllocator(itemToRelease, RemoveContext::kEviction, - /* isNascent */ movedToNextTier, candidate); + const auto res = releaseBackToAllocator(itemToRelease, RemoveContext::kEviction, + /* isNascent */ movedToNextTier); + XDCHECK(res == ReleaseRes::kReleased); } diff --git a/cachelib/allocator/FreeThresholdStrategy.cpp b/cachelib/allocator/FreeThresholdStrategy.cpp index efbbfbf1ee..b02d4bd354 100644 --- a/cachelib/allocator/FreeThresholdStrategy.cpp +++ b/cachelib/allocator/FreeThresholdStrategy.cpp @@ -32,14 +32,19 @@ bool FreeThresholdStrategy::shouldEvict(const CacheBase& cache, ClassId cid ) { const auto& mpStats = cache.getPoolByTid(pid,tid).getStats(); - size_t totalMem = mpStats.acStats.at(cid).getTotalMemory(); - size_t freeMem = mpStats.acStats.at(cid).getTotalFreeMemory(); + size_t allocSize = mpStats.acStats.at(cid).allocSize; + size_t totalMem = mpStats.acStats.at(cid).getTotalMemory() / allocSize; + size_t freeMem = mpStats.acStats.at(cid).getTotalFreeMemory() / allocSize; - double currFree = (double)freeMem/(double)totalMem; - if (currFree < freeThreshold_) { - return true; + if (totalMem > 0) { + double currFree = (double)freeMem/(double)totalMem; + if (currFree < freeThreshold_) { + return true; + } else { + return false; + } } else { - return false; + return false; } } diff --git a/cachelib/allocator/KeepFreeStrategy.cpp b/cachelib/allocator/KeepFreeStrategy.cpp index b894ab1f92..037e2067a1 100644 --- a/cachelib/allocator/KeepFreeStrategy.cpp +++ b/cachelib/allocator/KeepFreeStrategy.cpp @@ -29,14 +29,16 @@ bool KeepFreeStrategy::shouldEvict(const CacheBase& cache, unsigned int tid, PoolId pid, ClassId cid ) { - const auto& mpStats = cache.getPoolByTid(pid,tid).getStats(); - size_t allocSize = mpStats.acStats.at(cid).allocSize; - size_t freeAllocs = mpStats.acStats.at(cid).getTotalFreeMemory() / allocSize; - if (freeAllocs < nKeepFree_) { - return true; - } else { - return false; - } + + const auto& mpStats = cache.getPoolByTid(pid,tid).getStats(); + size_t allocSize = mpStats.acStats.at(cid).allocSize; + size_t totalAllocs = mpStats.acStats.at(cid).getTotalMemory() / allocSize; + size_t freeAllocs = mpStats.acStats.at(cid).getTotalFreeMemory() / allocSize; + if (totalAllocs > nKeepFree_ && freeAllocs < nKeepFree_) { + return true; + } else { + return false; + } } From e2263759b104c2590891a3109e8a72444292a4c2 Mon Sep 17 00:00:00 2001 From: Daniel Byrne Date: Wed, 23 Mar 2022 10:31:13 -0400 Subject: [PATCH 07/13] updated failing tests + review comments --- cachelib/allocator/CacheAllocator-inl.h | 6 ------ cachelib/allocator/FreeThresholdStrategy.cpp | 13 +++---------- cachelib/allocator/tests/CacheBaseTest.cpp | 2 ++ 3 files changed, 5 insertions(+), 16 deletions(-) diff --git a/cachelib/allocator/CacheAllocator-inl.h b/cachelib/allocator/CacheAllocator-inl.h index 0ac4057e8c..a8cf46f49a 100644 --- a/cachelib/allocator/CacheAllocator-inl.h +++ b/cachelib/allocator/CacheAllocator-inl.h @@ -319,12 +319,6 @@ void CacheAllocator::initWorkers() { startNewReaper(config_.reaperInterval, config_.reaperConfig); } - if (config_.poolOptimizerEnabled()) { - startNewPoolOptimizer(config_.regularPoolOptimizeInterval, - config_.compactCacheOptimizeInterval, - config_.poolOptimizeStrategy, - config_.ccacheOptimizeStepSizePercent); - } if (config_.poolOptimizerEnabled()) { startNewPoolOptimizer(config_.regularPoolOptimizeInterval, config_.compactCacheOptimizeInterval, diff --git a/cachelib/allocator/FreeThresholdStrategy.cpp b/cachelib/allocator/FreeThresholdStrategy.cpp index b02d4bd354..8c10166191 100644 --- a/cachelib/allocator/FreeThresholdStrategy.cpp +++ b/cachelib/allocator/FreeThresholdStrategy.cpp @@ -34,18 +34,11 @@ bool FreeThresholdStrategy::shouldEvict(const CacheBase& cache, const auto& mpStats = cache.getPoolByTid(pid,tid).getStats(); size_t allocSize = mpStats.acStats.at(cid).allocSize; size_t totalMem = mpStats.acStats.at(cid).getTotalMemory() / allocSize; - size_t freeMem = mpStats.acStats.at(cid).getTotalFreeMemory() / allocSize; - if (totalMem > 0) { - double currFree = (double)freeMem/(double)totalMem; - if (currFree < freeThreshold_) { - return true; - } else { - return false; - } - } else { - return false; + size_t freeMem = mpStats.acStats.at(cid).getTotalFreeMemory() / allocSize; + return ((double)freeMem / (double)totalMem) < freeThreshold_; } + return false; } diff --git a/cachelib/allocator/tests/CacheBaseTest.cpp b/cachelib/allocator/tests/CacheBaseTest.cpp index 7818034173..3a6b082084 100644 --- a/cachelib/allocator/tests/CacheBaseTest.cpp +++ b/cachelib/allocator/tests/CacheBaseTest.cpp @@ -32,6 +32,8 @@ class CacheBaseTest : public CacheBase, public SlabAllocatorTestBase { memoryPool_(0, 1024, *slabAllocator_, {64}) {} const std::string getCacheName() const override { return cacheName; } const MemoryPool& getPool(PoolId) const override { return memoryPool_; } + //TODO: do we support tiers in CacheBaseTEst + const MemoryPool& getPoolByTid(PoolId, TierId tid) const override { return memoryPool_; } PoolStats getPoolStats(PoolId) const override { return PoolStats(); } AllSlabReleaseEvents getAllSlabReleaseEvents(PoolId) const override { return AllSlabReleaseEvents{}; From 8401d08a31d1b5f2769fbaa2d484678ed6fda3bb Mon Sep 17 00:00:00 2001 From: Daniel Byrne Date: Wed, 23 Mar 2022 11:29:58 -0400 Subject: [PATCH 08/13] one more edit --- cachelib/allocator/KeepFreeStrategy.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/cachelib/allocator/KeepFreeStrategy.cpp b/cachelib/allocator/KeepFreeStrategy.cpp index 037e2067a1..99aa8005d9 100644 --- a/cachelib/allocator/KeepFreeStrategy.cpp +++ b/cachelib/allocator/KeepFreeStrategy.cpp @@ -34,12 +34,10 @@ bool KeepFreeStrategy::shouldEvict(const CacheBase& cache, size_t allocSize = mpStats.acStats.at(cid).allocSize; size_t totalAllocs = mpStats.acStats.at(cid).getTotalMemory() / allocSize; size_t freeAllocs = mpStats.acStats.at(cid).getTotalFreeMemory() / allocSize; - if (totalAllocs > nKeepFree_ && freeAllocs < nKeepFree_) { + if (freeAllocs < nKeepFree_) { return true; - } else { - return false; } - + return false; } unsigned int KeepFreeStrategy::calculateBatchSize(const CacheBase& cache, From 1b4a9864514f64d6fb9858f9ae2f1d75e41f0d3a Mon Sep 17 00:00:00 2001 From: Daniel Byrne Date: Mon, 28 Mar 2022 20:04:44 -0400 Subject: [PATCH 09/13] polling vs. tasks options --- cachelib/allocator/BackgroundEvictor-inl.h | 30 +++++++++---- cachelib/allocator/BackgroundEvictor.h | 7 +++- .../allocator/BackgroundEvictorStrategy.h | 13 ++---- .../allocator/BackgroundEvictorStrategy.old | 42 +++++++++++++++++++ cachelib/allocator/CacheAllocator-inl.h | 1 + cachelib/allocator/CacheAllocator.h | 8 +--- cachelib/allocator/FreeThresholdStrategy.cpp | 4 +- cachelib/allocator/FreeThresholdStrategy.h | 2 +- cachelib/allocator/KeepFreeStrategy.cpp | 4 +- cachelib/allocator/KeepFreeStrategy.h | 2 +- cachelib/cachebench/util/CacheConfig.cpp | 7 ++-- cachelib/cachebench/util/CacheConfig.h | 1 + 12 files changed, 88 insertions(+), 33 deletions(-) create mode 100644 cachelib/allocator/BackgroundEvictorStrategy.old diff --git a/cachelib/allocator/BackgroundEvictor-inl.h b/cachelib/allocator/BackgroundEvictor-inl.h index 2cc8538e59..c1e5697fdf 100644 --- a/cachelib/allocator/BackgroundEvictor-inl.h +++ b/cachelib/allocator/BackgroundEvictor-inl.h @@ -35,15 +35,31 @@ BackgroundEvictor::~BackgroundEvictor() { stop(std::chrono::seconds(0)); template void BackgroundEvictor::work() { - try { - for (const auto pid : cache_.getRegularPoolIds()) { - //check if the pool is full - probably should be if tier is full - if (cache_.getPoolByTid(pid,tid_).allSlabsAllocated()) { - checkAndRun(pid); + if (strategy_->poll_) { + try { + for (const auto pid : cache_.getRegularPoolIds()) { + //check if the pool is full - probably should be if tier is full + if (cache_.getPoolByTid(pid,tid_).allSlabsAllocated()) { + checkAndRun(pid); + } } + } catch (const std::exception& ex) { + XLOGF(ERR, "BackgroundEvictor interrupted due to exception: {}", ex.what()); } - } catch (const std::exception& ex) { - XLOGF(ERR, "BackgroundEvictor interrupted due to exception: {}", ex.what()); + } else { + //when an eviction for a given pid,cid at tier 0 is triggered this will be run + while (1) { + std::pair p = tasks_.dequeue(); + unsigned int pid = p.first; + unsigned int cid = p.second; + unsigned int batch = strategy_->calculateBatchSize(cache_,tid_,pid,cid); + //try evicting BATCH items from the class in order to reach free target + unsigned int evicted = + BackgroundEvictorAPIWrapper::traverseAndEvictItems(cache_, + tid_,pid,cid,batch); + runCount_ = runCount_ + 1; + } + } } diff --git a/cachelib/allocator/BackgroundEvictor.h b/cachelib/allocator/BackgroundEvictor.h index b53b0f7593..082d23da31 100644 --- a/cachelib/allocator/BackgroundEvictor.h +++ b/cachelib/allocator/BackgroundEvictor.h @@ -17,6 +17,7 @@ #pragma once #include +#include #include "cachelib/allocator/CacheStats.h" #include "cachelib/common/PeriodicWorker.h" @@ -53,7 +54,10 @@ class BackgroundEvictor : public PeriodicWorker { unsigned int tid); ~BackgroundEvictor() override; - + + void schedule(size_t pid, size_t cid) { + tasks_.enqueue(std::make_pair(pid,cid)); + } BackgroundEvictorStats getStats() const noexcept; private: @@ -64,6 +68,7 @@ class BackgroundEvictor : public PeriodicWorker { Cache& cache_; std::shared_ptr strategy_; unsigned int tid_; + folly::UMPSCQueue,true> tasks_; // implements the actual logic of running the background evictor void work() override final; diff --git a/cachelib/allocator/BackgroundEvictorStrategy.h b/cachelib/allocator/BackgroundEvictorStrategy.h index ae053ee9b5..7ae978e5bf 100644 --- a/cachelib/allocator/BackgroundEvictorStrategy.h +++ b/cachelib/allocator/BackgroundEvictorStrategy.h @@ -21,21 +21,14 @@ namespace facebook { namespace cachelib { -//struct RebalanceContext { -// // Victim and Receiver must belong to the same pool -// ClassId victimClassId{Slab::kInvalidClassId}; -// ClassId receiverClassId{Slab::kInvalidClassId}; -// -// RebalanceContext() = default; -// RebalanceContext(ClassId victim, ClassId receiver) -// : victimClassId(victim), receiverClassId(receiver) {} -//}; // Base class for background eviction strategy. class BackgroundEvictorStrategy { public: + BackgroundEvictorStrategy(bool poll) : poll_{poll} {}; + virtual unsigned int calculateBatchSize(const CacheBase& cache, unsigned int tid, PoolId pid, @@ -46,6 +39,8 @@ class BackgroundEvictorStrategy { PoolId pid, ClassId cid ) = 0; + // if we should poll every n ms or wait for tasks in queue + bool poll_{false}; }; diff --git a/cachelib/allocator/BackgroundEvictorStrategy.old b/cachelib/allocator/BackgroundEvictorStrategy.old new file mode 100644 index 0000000000..0cb7218c22 --- /dev/null +++ b/cachelib/allocator/BackgroundEvictorStrategy.old @@ -0,0 +1,42 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * 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 "cachelib/allocator/BackgroundEvictorStrategy.h" + +#include + +namespace facebook { +namespace cachelib { + + +bool BackgroundEvictorStrategy::shouldEvict(const CacheBase& cache, + unsigned int tid, + PoolId pid, + ClassId cid ) { + return true; +} + + +unsigned int BackgroundEvictorStrategy::calculateBatchSize(const CacheBase& cache, + unsigned int tid, + PoolId pid, + ClassId cid ) { + return 1; +} + + +} // namespace cachelib +} // namespace facebook diff --git a/cachelib/allocator/CacheAllocator-inl.h b/cachelib/allocator/CacheAllocator-inl.h index a8cf46f49a..1ee4e6fded 100644 --- a/cachelib/allocator/CacheAllocator-inl.h +++ b/cachelib/allocator/CacheAllocator-inl.h @@ -382,6 +382,7 @@ CacheAllocator::allocateInternalTier(TierId tid, // Should we support eviction between memory tiers (e.g. from DRAM to PMEM)? if (memory == nullptr && !config_.disableEviction) { memory = findEviction(tid, pid, cid); + backgroundEvictor_->schedule(pid,cid); } ItemHandle handle; diff --git a/cachelib/allocator/CacheAllocator.h b/cachelib/allocator/CacheAllocator.h index af6069e3a5..9207075aff 100644 --- a/cachelib/allocator/CacheAllocator.h +++ b/cachelib/allocator/CacheAllocator.h @@ -1789,11 +1789,6 @@ class CacheAllocator : public CacheBase { if(toReleaseHandle) { movedToNextTier = true; } else { - //toReleaseHandle = - // itr->isChainedItem() - // ? advanceIteratorAndTryEvictChainedItem(tid, pid, itr) - // : advanceIteratorAndTryEvictRegularItem(tid, pid, mmContainer, itr); - //just bump the iterator and move on with life ++itr; } @@ -1805,7 +1800,6 @@ class CacheAllocator : public CacheBase { } ++evictions; - // we must be the last handle and for chained items, this will be // the parent. XDCHECK(toReleaseHandle.get() == candidate || candidate->isChainedItem()); @@ -1816,7 +1810,7 @@ class CacheAllocator : public CacheBase { // an already zero refcount, which will throw exception auto& itemToRelease = *toReleaseHandle.release(); - // Decrementing the refcount because we want to recycle the item + // Decrementing the refcount because we will call releaseBackToAllocator const auto ref = decRef(itemToRelease); XDCHECK_EQ(0u, ref); diff --git a/cachelib/allocator/FreeThresholdStrategy.cpp b/cachelib/allocator/FreeThresholdStrategy.cpp index 8c10166191..1fc144394e 100644 --- a/cachelib/allocator/FreeThresholdStrategy.cpp +++ b/cachelib/allocator/FreeThresholdStrategy.cpp @@ -23,8 +23,8 @@ namespace cachelib { -FreeThresholdStrategy::FreeThresholdStrategy(double freeThreshold) - : freeThreshold_(freeThreshold) {} +FreeThresholdStrategy::FreeThresholdStrategy(double freeThreshold, bool poll) + : BackgroundEvictorStrategy(poll), freeThreshold_(freeThreshold) {} bool FreeThresholdStrategy::shouldEvict(const CacheBase& cache, unsigned int tid, diff --git a/cachelib/allocator/FreeThresholdStrategy.h b/cachelib/allocator/FreeThresholdStrategy.h index e0954e888d..c424702988 100644 --- a/cachelib/allocator/FreeThresholdStrategy.h +++ b/cachelib/allocator/FreeThresholdStrategy.h @@ -28,7 +28,7 @@ class FreeThresholdStrategy : public BackgroundEvictorStrategy { public: - FreeThresholdStrategy(double freeThreshold); + FreeThresholdStrategy(double freeThreshold, bool poll); ~FreeThresholdStrategy() {} diff --git a/cachelib/allocator/KeepFreeStrategy.cpp b/cachelib/allocator/KeepFreeStrategy.cpp index 99aa8005d9..adc7d274ea 100644 --- a/cachelib/allocator/KeepFreeStrategy.cpp +++ b/cachelib/allocator/KeepFreeStrategy.cpp @@ -22,8 +22,8 @@ namespace facebook { namespace cachelib { -KeepFreeStrategy::KeepFreeStrategy(unsigned int nKeepFree) - : nKeepFree_(nKeepFree) {} +KeepFreeStrategy::KeepFreeStrategy(unsigned int nKeepFree, bool poll) + : BackgroundEvictorStrategy(poll), nKeepFree_(nKeepFree) {} bool KeepFreeStrategy::shouldEvict(const CacheBase& cache, unsigned int tid, diff --git a/cachelib/allocator/KeepFreeStrategy.h b/cachelib/allocator/KeepFreeStrategy.h index 70fcabbeab..8da6277ebd 100644 --- a/cachelib/allocator/KeepFreeStrategy.h +++ b/cachelib/allocator/KeepFreeStrategy.h @@ -26,7 +26,7 @@ class KeepFreeStrategy : public BackgroundEvictorStrategy { public: - KeepFreeStrategy(unsigned int nKeepFree); + KeepFreeStrategy(unsigned int nKeepFree, bool poll); ~KeepFreeStrategy() {} diff --git a/cachelib/cachebench/util/CacheConfig.cpp b/cachelib/cachebench/util/CacheConfig.cpp index bd0ae5d3b8..e3f6e985af 100644 --- a/cachelib/cachebench/util/CacheConfig.cpp +++ b/cachelib/cachebench/util/CacheConfig.cpp @@ -37,6 +37,7 @@ CacheConfig::CacheConfig(const folly::dynamic& configJson) { JSONSetVal(configJson, rebalanceDiffRatio); JSONSetVal(configJson, backgroundEvictorStrategy); + JSONSetVal(configJson, backgroundEvictorPoll); JSONSetVal(configJson, freeThreshold); JSONSetVal(configJson, nKeepFree); @@ -148,12 +149,12 @@ std::shared_ptr CacheConfig::getBackgroundEvictorStra } if (backgroundEvictorStrategy == "free-threshold") { - return std::make_shared(freeThreshold); + return std::make_shared(freeThreshold,backgroundEvictorPoll); } else if (backgroundEvictorStrategy == "keep-free") { - return std::make_shared(nKeepFree); + return std::make_shared(nKeepFree,backgroundEvictorPoll); } else { //default! - return std::make_shared(freeThreshold); + return std::make_shared(freeThreshold,true); } } diff --git a/cachelib/cachebench/util/CacheConfig.h b/cachelib/cachebench/util/CacheConfig.h index e51470555f..d61aa5e889 100644 --- a/cachelib/cachebench/util/CacheConfig.h +++ b/cachelib/cachebench/util/CacheConfig.h @@ -77,6 +77,7 @@ struct CacheConfig : public JSONConfig { std::string backgroundEvictorStrategy; double freeThreshold{0.01}; //keep 1% of space free uint32_t nKeepFree{100}; //keep at most 100 slots free + bool backgroundEvictorPoll{false}; uint64_t rebalanceMinSlabs{1}; double rebalanceDiffRatio{0.25}; bool moveOnSlabRelease{false}; From 616a179480b202a38172238c84eddec7be229907 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Igor=20Chor=C4=85=C5=BCewicz?= Date: Thu, 31 Mar 2022 18:59:59 +0200 Subject: [PATCH 10/13] Simplify BG evictor code and add possibility to wakeup it --- cachelib/allocator/BackgroundEvictor-inl.h | 56 +++++++++---------- cachelib/allocator/BackgroundEvictor.h | 5 +- .../allocator/BackgroundEvictorStrategy.h | 14 +---- .../allocator/BackgroundEvictorStrategy.old | 42 -------------- cachelib/allocator/CacheAllocator-inl.h | 9 ++- cachelib/allocator/CacheAllocator.h | 20 ++----- cachelib/allocator/CacheAllocatorConfig.h | 6 ++ cachelib/allocator/CacheStats.h | 5 +- cachelib/allocator/FreeThresholdStrategy.cpp | 34 ++--------- cachelib/allocator/FreeThresholdStrategy.h | 15 +---- cachelib/allocator/KeepFreeStrategy.cpp | 29 ++-------- cachelib/allocator/KeepFreeStrategy.h | 15 +---- cachelib/cachebench/cache/Cache-inl.h | 4 ++ cachelib/cachebench/cache/CacheStats.h | 2 + cachelib/cachebench/util/CacheConfig.cpp | 10 ++-- cachelib/cachebench/util/CacheConfig.h | 4 +- 16 files changed, 84 insertions(+), 186 deletions(-) delete mode 100644 cachelib/allocator/BackgroundEvictorStrategy.old diff --git a/cachelib/allocator/BackgroundEvictor-inl.h b/cachelib/allocator/BackgroundEvictor-inl.h index c1e5697fdf..5908477479 100644 --- a/cachelib/allocator/BackgroundEvictor-inl.h +++ b/cachelib/allocator/BackgroundEvictor-inl.h @@ -27,7 +27,6 @@ BackgroundEvictor::BackgroundEvictor(Cache& cache, : cache_(cache), strategy_(strategy), tid_(tid) { - } template @@ -35,52 +34,46 @@ BackgroundEvictor::~BackgroundEvictor() { stop(std::chrono::seconds(0)); template void BackgroundEvictor::work() { - if (strategy_->poll_) { - try { + try { + if (!tasks_.empty()) { + while (auto entry = tasks_.try_dequeue()) { + auto [pid, cid] = entry.value(); + auto batch = strategy_->calculateBatchSize(cache_, tid_, pid, cid); + auto evicted = BackgroundEvictorAPIWrapper::traverseAndEvictItems(cache_, + tid_,pid,cid,batch); + numEvictedItemsFromSchedule_.fetch_add(1, std::memory_order_relaxed); + runCount_.fetch_add(1, std::memory_order_relaxed); + } + } else { for (const auto pid : cache_.getRegularPoolIds()) { //check if the pool is full - probably should be if tier is full if (cache_.getPoolByTid(pid,tid_).allSlabsAllocated()) { checkAndRun(pid); } } - } catch (const std::exception& ex) { - XLOGF(ERR, "BackgroundEvictor interrupted due to exception: {}", ex.what()); } - } else { - //when an eviction for a given pid,cid at tier 0 is triggered this will be run - while (1) { - std::pair p = tasks_.dequeue(); - unsigned int pid = p.first; - unsigned int cid = p.second; - unsigned int batch = strategy_->calculateBatchSize(cache_,tid_,pid,cid); - //try evicting BATCH items from the class in order to reach free target - unsigned int evicted = - BackgroundEvictorAPIWrapper::traverseAndEvictItems(cache_, - tid_,pid,cid,batch); - runCount_ = runCount_ + 1; - } - + } catch (const std::exception& ex) { + XLOGF(ERR, "BackgroundEvictor interrupted due to exception: {}", ex.what()); } } - // Look for classes that exceed the target memory capacity // and return those for eviction template -void BackgroundEvictor::checkAndRun(PoolId pid) { - +void BackgroundEvictor::checkAndRun(PoolId pid) { const auto& mpStats = cache_.getPoolByTid(pid,tid_).getStats(); for (auto& cid : mpStats.classIds) { - if (strategy_->shouldEvict(cache_,tid_,pid,cid)) { - unsigned int batch = strategy_->calculateBatchSize(cache_,tid_,pid,cid); - //try evicting BATCH items from the class in order to reach free target - unsigned int evicted = - BackgroundEvictorAPIWrapper::traverseAndEvictItems(cache_, - tid_,pid,cid,batch); - numEvictedItems_ += evicted; - } + auto batch = strategy_->calculateBatchSize(cache_,tid_,pid,cid); + if (!batch) + continue; + + //try evicting BATCH items from the class in order to reach free target + auto evicted = + BackgroundEvictorAPIWrapper::traverseAndEvictItems(cache_, + tid_,pid,cid,batch); + numEvictedItems_.fetch_add(evicted, std::memory_order_relaxed); } - runCount_ = runCount_ + 1; + runCount_.fetch_add(1, std::memory_order_relaxed); } template @@ -88,6 +81,7 @@ BackgroundEvictorStats BackgroundEvictor::getStats() const noexcept { BackgroundEvictorStats stats; stats.numEvictedItems = numEvictedItems_.load(std::memory_order_relaxed); stats.numTraversals = runCount_.load(std::memory_order_relaxed); + stats.numEvictedItemsFromSchedule = numEvictedItemsFromSchedule_.load(std::memory_order_relaxed); return stats; } diff --git a/cachelib/allocator/BackgroundEvictor.h b/cachelib/allocator/BackgroundEvictor.h index 082d23da31..bbcc11add0 100644 --- a/cachelib/allocator/BackgroundEvictor.h +++ b/cachelib/allocator/BackgroundEvictor.h @@ -32,8 +32,8 @@ namespace cachelib { template struct BackgroundEvictorAPIWrapper { - static unsigned int traverseAndEvictItems(C& cache, - unsigned int tid, unsigned int pid, unsigned int cid, unsigned int batch) { + static size_t traverseAndEvictItems(C& cache, + unsigned int tid, unsigned int pid, unsigned int cid, size_t batch) { return cache.traverseAndEvictItems(tid,pid,cid,batch); } }; @@ -75,6 +75,7 @@ class BackgroundEvictor : public PeriodicWorker { void checkAndRun(PoolId pid); std::atomic numEvictedItems_{0}; + std::atomic numEvictedItemsFromSchedule_{0}; std::atomic runCount_{0}; }; } // namespace cachelib diff --git a/cachelib/allocator/BackgroundEvictorStrategy.h b/cachelib/allocator/BackgroundEvictorStrategy.h index 7ae978e5bf..00dd3e878b 100644 --- a/cachelib/allocator/BackgroundEvictorStrategy.h +++ b/cachelib/allocator/BackgroundEvictorStrategy.h @@ -26,22 +26,10 @@ namespace cachelib { class BackgroundEvictorStrategy { public: - - BackgroundEvictorStrategy(bool poll) : poll_{poll} {}; - - virtual unsigned int calculateBatchSize(const CacheBase& cache, - unsigned int tid, - PoolId pid, - ClassId cid ) = 0; - - virtual bool shouldEvict(const CacheBase& cache, + virtual size_t calculateBatchSize(const CacheBase& cache, unsigned int tid, PoolId pid, ClassId cid ) = 0; - - // if we should poll every n ms or wait for tasks in queue - bool poll_{false}; - }; } // namespace cachelib diff --git a/cachelib/allocator/BackgroundEvictorStrategy.old b/cachelib/allocator/BackgroundEvictorStrategy.old deleted file mode 100644 index 0cb7218c22..0000000000 --- a/cachelib/allocator/BackgroundEvictorStrategy.old +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * 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 "cachelib/allocator/BackgroundEvictorStrategy.h" - -#include - -namespace facebook { -namespace cachelib { - - -bool BackgroundEvictorStrategy::shouldEvict(const CacheBase& cache, - unsigned int tid, - PoolId pid, - ClassId cid ) { - return true; -} - - -unsigned int BackgroundEvictorStrategy::calculateBatchSize(const CacheBase& cache, - unsigned int tid, - PoolId pid, - ClassId cid ) { - return 1; -} - - -} // namespace cachelib -} // namespace facebook diff --git a/cachelib/allocator/CacheAllocator-inl.h b/cachelib/allocator/CacheAllocator-inl.h index 1ee4e6fded..0761b0a7c6 100644 --- a/cachelib/allocator/CacheAllocator-inl.h +++ b/cachelib/allocator/CacheAllocator-inl.h @@ -382,7 +382,14 @@ CacheAllocator::allocateInternalTier(TierId tid, // Should we support eviction between memory tiers (e.g. from DRAM to PMEM)? if (memory == nullptr && !config_.disableEviction) { memory = findEviction(tid, pid, cid); - backgroundEvictor_->schedule(pid,cid); + + if (backgroundEvictor_ && config_.scheduleEviction) { + backgroundEvictor_->schedule(pid,cid); + } + + if (backgroundEvictor_ && config_.wakeupBgEvictor) { + backgroundEvictor_->wakeUp(); + } } ItemHandle handle; diff --git a/cachelib/allocator/CacheAllocator.h b/cachelib/allocator/CacheAllocator.h index 9207075aff..4ae045b77d 100644 --- a/cachelib/allocator/CacheAllocator.h +++ b/cachelib/allocator/CacheAllocator.h @@ -1769,13 +1769,12 @@ class CacheAllocator : public CacheBase { // exposed for the background evictor to iterate through the memory and evict // in batch. This should improve insertion path for tiered memory config - unsigned int traverseAndEvictItems(unsigned int tid, unsigned int pid, unsigned int cid, unsigned int batch) { - + size_t traverseAndEvictItems(unsigned int tid, unsigned int pid, unsigned int cid, size_t batch) { auto& mmContainer = getMMContainer(tid, pid, cid); - unsigned int evictions = 0; + size_t evictions = 0; auto itr = mmContainer.getEvictionIterator(); - while (evictions < batch && itr) { + while (evictions < batch && itr) { Item* candidate = itr.get(); if (candidate == NULL) { break; @@ -1785,14 +1784,9 @@ class CacheAllocator : public CacheBase { // recycles the child we intend to. ItemHandle toReleaseHandle = tryEvictToNextMemoryTier(tid, pid, itr); - bool movedToNextTier = false; - if(toReleaseHandle) { - movedToNextTier = true; - } else { + if (!toReleaseHandle) { ++itr; - } - - if (toReleaseHandle) { + } else { if (toReleaseHandle->hasChainedItem()) { (*stats_.chainedItemEvictions)[pid][cid].inc(); } else { @@ -1817,11 +1811,9 @@ class CacheAllocator : public CacheBase { // check if by releasing the item we intend to, we actually // recycle the candidate. const auto res = releaseBackToAllocator(itemToRelease, RemoveContext::kEviction, - /* isNascent */ movedToNextTier); + /* isNascent */ true); XDCHECK(res == ReleaseRes::kReleased); - } - } // Invalidate iterator since later on we may use this mmContainer diff --git a/cachelib/allocator/CacheAllocatorConfig.h b/cachelib/allocator/CacheAllocatorConfig.h index 97eec11411..56e817ddef 100644 --- a/cachelib/allocator/CacheAllocatorConfig.h +++ b/cachelib/allocator/CacheAllocatorConfig.h @@ -597,6 +597,12 @@ class CacheAllocatorConfig { // skip promote children items in chained when parent fail to promote bool skipPromoteChildrenWhenParentFailed{false}; + // wakeupBg evictor each time there is no memory to allocate new item in topmost tier + bool wakeupBgEvictor {false}; + + // every time there is no space to allocate for particual cid,pid pass this information to BG worker + bool scheduleEviction {false}; + friend CacheT; private: diff --git a/cachelib/allocator/CacheStats.h b/cachelib/allocator/CacheStats.h index 2da0e82d57..dbda45ebae 100644 --- a/cachelib/allocator/CacheStats.h +++ b/cachelib/allocator/CacheStats.h @@ -287,9 +287,12 @@ struct ReaperStats { // Stats for background evictor struct BackgroundEvictorStats { - // the total number of items the reaper has visited. + // the number of items this worker evicted by looking at pools/classes stats uint64_t numEvictedItems{0}; + // the number of items this worker evicted for pools/classes requested by schedule call + uint64_t numEvictedItemsFromSchedule{0}; + // number of times we went executed the thread //TODO: is this def correct? uint64_t numTraversals{0}; diff --git a/cachelib/allocator/FreeThresholdStrategy.cpp b/cachelib/allocator/FreeThresholdStrategy.cpp index 1fc144394e..acafc9d05b 100644 --- a/cachelib/allocator/FreeThresholdStrategy.cpp +++ b/cachelib/allocator/FreeThresholdStrategy.cpp @@ -23,39 +23,17 @@ namespace cachelib { -FreeThresholdStrategy::FreeThresholdStrategy(double freeThreshold, bool poll) - : BackgroundEvictorStrategy(poll), freeThreshold_(freeThreshold) {} +FreeThresholdStrategy::FreeThresholdStrategy(double freeThreshold) + : freeThreshold_(freeThreshold) {} -bool FreeThresholdStrategy::shouldEvict(const CacheBase& cache, +size_t FreeThresholdStrategy::calculateBatchSize(const CacheBase& cache, unsigned int tid, PoolId pid, ClassId cid ) { - - const auto& mpStats = cache.getPoolByTid(pid,tid).getStats(); - size_t allocSize = mpStats.acStats.at(cid).allocSize; - size_t totalMem = mpStats.acStats.at(cid).getTotalMemory() / allocSize; - if (totalMem > 0) { - size_t freeMem = mpStats.acStats.at(cid).getTotalFreeMemory() / allocSize; - return ((double)freeMem / (double)totalMem) < freeThreshold_; - } - return false; - -} - -unsigned int FreeThresholdStrategy::calculateBatchSize(const CacheBase& cache, - unsigned int tid, - PoolId pid, - ClassId cid ) { - const auto& mpStats = cache.getPoolByTid(pid,tid).getStats(); - size_t totalMem = mpStats.acStats.at(cid).getTotalMemory(); - size_t freeMem = mpStats.acStats.at(cid).getTotalFreeMemory(); - - size_t targetMem = (freeThreshold_ * totalMem) - freeMem; - unsigned int batch = (targetMem / mpStats.acStats.at(cid).allocSize); - - return batch; + const auto& mpStats = cache.getPoolByTid(pid,tid).getStats().acStats.at(cid); + size_t targetMem = (freeThreshold_ * mpStats.getTotalMemory()) - mpStats.getTotalFreeMemory(); + return std::max(0UL, targetMem / mpStats.allocSize); } - } // namespace cachelib } // namespace facebook diff --git a/cachelib/allocator/FreeThresholdStrategy.h b/cachelib/allocator/FreeThresholdStrategy.h index c424702988..853d72f25b 100644 --- a/cachelib/allocator/FreeThresholdStrategy.h +++ b/cachelib/allocator/FreeThresholdStrategy.h @@ -27,26 +27,15 @@ namespace cachelib { class FreeThresholdStrategy : public BackgroundEvictorStrategy { public: - - FreeThresholdStrategy(double freeThreshold, bool poll); - + FreeThresholdStrategy(double freeThreshold); ~FreeThresholdStrategy() {} - unsigned int calculateBatchSize(const CacheBase& cache, - unsigned int tid, - PoolId pid, - ClassId cid ); - - bool shouldEvict(const CacheBase& cache, + size_t calculateBatchSize(const CacheBase& cache, unsigned int tid, PoolId pid, ClassId cid ); - private: - double freeThreshold_; - - }; } // namespace cachelib diff --git a/cachelib/allocator/KeepFreeStrategy.cpp b/cachelib/allocator/KeepFreeStrategy.cpp index adc7d274ea..fe291b45f9 100644 --- a/cachelib/allocator/KeepFreeStrategy.cpp +++ b/cachelib/allocator/KeepFreeStrategy.cpp @@ -22,35 +22,16 @@ namespace facebook { namespace cachelib { -KeepFreeStrategy::KeepFreeStrategy(unsigned int nKeepFree, bool poll) - : BackgroundEvictorStrategy(poll), nKeepFree_(nKeepFree) {} +KeepFreeStrategy::KeepFreeStrategy(size_t nKeepFree) + : nKeepFree_(nKeepFree) {} -bool KeepFreeStrategy::shouldEvict(const CacheBase& cache, +size_t KeepFreeStrategy::calculateBatchSize(const CacheBase& cache, unsigned int tid, PoolId pid, ClassId cid ) { - - const auto& mpStats = cache.getPoolByTid(pid,tid).getStats(); - size_t allocSize = mpStats.acStats.at(cid).allocSize; - size_t totalAllocs = mpStats.acStats.at(cid).getTotalMemory() / allocSize; - size_t freeAllocs = mpStats.acStats.at(cid).getTotalFreeMemory() / allocSize; - if (freeAllocs < nKeepFree_) { - return true; - } - return false; -} - -unsigned int KeepFreeStrategy::calculateBatchSize(const CacheBase& cache, - unsigned int tid, - PoolId pid, - ClassId cid ) { - const auto& mpStats = cache.getPoolByTid(pid,tid).getStats(); - size_t allocSize = mpStats.acStats.at(cid).allocSize; - size_t freeAllocs = mpStats.acStats.at(cid).getTotalFreeMemory() / allocSize; - - return (unsigned int)(nKeepFree_ - freeAllocs); + const auto& mpStats = cache.getPoolByTid(pid,tid).getStats().acStats.at(cid); + return std::max(0UL, nKeepFree_ - (mpStats.getTotalFreeMemory() / mpStats.allocSize)); } - } // namespace cachelib } // namespace facebook diff --git a/cachelib/allocator/KeepFreeStrategy.h b/cachelib/allocator/KeepFreeStrategy.h index 8da6277ebd..bf3ec9a364 100644 --- a/cachelib/allocator/KeepFreeStrategy.h +++ b/cachelib/allocator/KeepFreeStrategy.h @@ -23,25 +23,16 @@ namespace facebook { namespace cachelib { class KeepFreeStrategy : public BackgroundEvictorStrategy { - public: - - KeepFreeStrategy(unsigned int nKeepFree, bool poll); - + KeepFreeStrategy(size_t nKeepFree); ~KeepFreeStrategy() {} - unsigned int calculateBatchSize(const CacheBase& cache, - unsigned int tid, - PoolId pid, - ClassId cid ); - - bool shouldEvict(const CacheBase& cache, + size_t calculateBatchSize(const CacheBase& cache, unsigned int tid, PoolId pid, ClassId cid ); private: - unsigned int nKeepFree_; - + size_t nKeepFree_; }; } // namespace cachelib diff --git a/cachelib/cachebench/cache/Cache-inl.h b/cachelib/cachebench/cache/Cache-inl.h index 2ada844d3d..75e207ded5 100644 --- a/cachelib/cachebench/cache/Cache-inl.h +++ b/cachelib/cachebench/cache/Cache-inl.h @@ -118,6 +118,9 @@ Cache::Cache(const CacheConfig& config, } }); + allocatorConfig_.wakeupBgEvictor = config_.wakeupBgEvictor; + allocatorConfig_.scheduleEviction = config_.scheduleEviction; + if (config_.enableItemDestructorCheck) { auto removeCB = [&](const typename Allocator::DestructorData& data) { if (!itemRecords_.validate(data)) { @@ -525,6 +528,7 @@ Stats Cache::getStats() const { ret.allocFailures = cacheStats.allocFailures; ret.numBackgroundEvictions = cacheStats.backgroundEvictorStats.numEvictedItems; + ret.numBackgroundEvictionsFromSchedule = cacheStats.backgroundEvictorStats.numEvictedItemsFromSchedule; ret.numBackgroundEvictorRuns = cacheStats.backgroundEvictorStats.numTraversals; ret.numCacheGets = cacheStats.numCacheGets; diff --git a/cachelib/cachebench/cache/CacheStats.h b/cachelib/cachebench/cache/CacheStats.h index 6285fcb3c2..b81affc942 100644 --- a/cachelib/cachebench/cache/CacheStats.h +++ b/cachelib/cachebench/cache/CacheStats.h @@ -33,6 +33,7 @@ struct Stats { uint64_t allocFailures{0}; uint64_t numBackgroundEvictions{0}; + uint64_t numBackgroundEvictionsFromSchedule{0}; uint64_t numBackgroundEvictorRuns{0}; uint64_t numCacheGets{0}; @@ -120,6 +121,7 @@ struct Stats { out << folly::sformat("Background Tier 0 Evictions : {:,}", numBackgroundEvictions) << std::endl; + out << folly::sformat("Background Tier 0 Evictions from schedule() : {:,}", numBackgroundEvictionsFromSchedule) << std::endl; out << folly::sformat("Background Tier 0 Eviction Runs : {:,}", numBackgroundEvictorRuns) << std::endl; diff --git a/cachelib/cachebench/util/CacheConfig.cpp b/cachelib/cachebench/util/CacheConfig.cpp index e3f6e985af..3a44a46188 100644 --- a/cachelib/cachebench/util/CacheConfig.cpp +++ b/cachelib/cachebench/util/CacheConfig.cpp @@ -37,7 +37,6 @@ CacheConfig::CacheConfig(const folly::dynamic& configJson) { JSONSetVal(configJson, rebalanceDiffRatio); JSONSetVal(configJson, backgroundEvictorStrategy); - JSONSetVal(configJson, backgroundEvictorPoll); JSONSetVal(configJson, freeThreshold); JSONSetVal(configJson, nKeepFree); @@ -104,6 +103,9 @@ CacheConfig::CacheConfig(const folly::dynamic& configJson) { JSONSetVal(configJson, persistedCacheDir); JSONSetVal(configJson, usePosixShm); + JSONSetVal(configJson, scheduleEviction); + JSONSetVal(configJson, wakeupBgEvictor); + if (configJson.count("memoryTiers")) { for (auto& it : configJson["memoryTiers"]) { memoryTierConfigs.push_back(MemoryTierConfig(it).getMemoryTierCacheConfig()); @@ -149,12 +151,12 @@ std::shared_ptr CacheConfig::getBackgroundEvictorStra } if (backgroundEvictorStrategy == "free-threshold") { - return std::make_shared(freeThreshold,backgroundEvictorPoll); + return std::make_shared(freeThreshold); } else if (backgroundEvictorStrategy == "keep-free") { - return std::make_shared(nKeepFree,backgroundEvictorPoll); + return std::make_shared(nKeepFree); } else { //default! - return std::make_shared(freeThreshold,true); + return std::make_shared(freeThreshold); } } diff --git a/cachelib/cachebench/util/CacheConfig.h b/cachelib/cachebench/util/CacheConfig.h index d61aa5e889..027ee67456 100644 --- a/cachelib/cachebench/util/CacheConfig.h +++ b/cachelib/cachebench/util/CacheConfig.h @@ -77,7 +77,6 @@ struct CacheConfig : public JSONConfig { std::string backgroundEvictorStrategy; double freeThreshold{0.01}; //keep 1% of space free uint32_t nKeepFree{100}; //keep at most 100 slots free - bool backgroundEvictorPoll{false}; uint64_t rebalanceMinSlabs{1}; double rebalanceDiffRatio{0.25}; bool moveOnSlabRelease{false}; @@ -289,6 +288,9 @@ struct CacheConfig : public JSONConfig { // this verifies whether the feature affects throughputs. bool enableItemDestructor{false}; + bool wakeupBgEvictor {false}; + bool scheduleEviction {false}; + explicit CacheConfig(const folly::dynamic& configJson); CacheConfig() {} From 69606affccf05e8d1629f594302dc3bef8ae65d3 Mon Sep 17 00:00:00 2001 From: Daniel Byrne Date: Wed, 13 Apr 2022 23:12:37 -0400 Subject: [PATCH 11/13] updated stats --- cachelib/allocator/BackgroundEvictor-inl.h | 25 +++++++++++++++++++--- cachelib/allocator/BackgroundEvictor.h | 5 ++++- cachelib/allocator/CacheAllocator.h | 5 +++++ cachelib/allocator/CacheStats.h | 6 +++++- cachelib/cachebench/cache/Cache-inl.h | 13 ++++++----- cachelib/cachebench/cache/CacheStats.h | 12 +++++++++++ 6 files changed, 56 insertions(+), 10 deletions(-) diff --git a/cachelib/allocator/BackgroundEvictor-inl.h b/cachelib/allocator/BackgroundEvictor-inl.h index 5908477479..8f246bcb63 100644 --- a/cachelib/allocator/BackgroundEvictor-inl.h +++ b/cachelib/allocator/BackgroundEvictor-inl.h @@ -62,28 +62,47 @@ void BackgroundEvictor::work() { template void BackgroundEvictor::checkAndRun(PoolId pid) { const auto& mpStats = cache_.getPoolByTid(pid,tid_).getStats(); + unsigned int evictions = 0; + unsigned int classes = 0; for (auto& cid : mpStats.classIds) { auto batch = strategy_->calculateBatchSize(cache_,tid_,pid,cid); - if (!batch) + if (!batch) { + classes++; continue; + } //try evicting BATCH items from the class in order to reach free target auto evicted = BackgroundEvictorAPIWrapper::traverseAndEvictItems(cache_, tid_,pid,cid,batch); - numEvictedItems_.fetch_add(evicted, std::memory_order_relaxed); + evictions += evicted; + const size_t cid_id = (size_t)cid; + auto it = evictions_per_class_.find(cid_id); + if (it != evictions_per_class_.end()) { + it->second += evicted; + } else { + evictions_per_class_[cid_id] = 0; + } } runCount_.fetch_add(1, std::memory_order_relaxed); + numEvictedItems_.fetch_add(evictions, std::memory_order_relaxed); + totalClasses_.fetch_add(classes, std::memory_order_relaxed); } template BackgroundEvictorStats BackgroundEvictor::getStats() const noexcept { BackgroundEvictorStats stats; stats.numEvictedItems = numEvictedItems_.load(std::memory_order_relaxed); - stats.numTraversals = runCount_.load(std::memory_order_relaxed); + stats.runCount = runCount_.load(std::memory_order_relaxed); stats.numEvictedItemsFromSchedule = numEvictedItemsFromSchedule_.load(std::memory_order_relaxed); + stats.totalClasses = totalClasses_.load(std::memory_order_relaxed); return stats; } +template +std::map BackgroundEvictor::getClassStats() const noexcept { + return evictions_per_class_; +} + } // namespace cachelib } // namespace facebook diff --git a/cachelib/allocator/BackgroundEvictor.h b/cachelib/allocator/BackgroundEvictor.h index bbcc11add0..3d8ac3a4cc 100644 --- a/cachelib/allocator/BackgroundEvictor.h +++ b/cachelib/allocator/BackgroundEvictor.h @@ -59,6 +59,7 @@ class BackgroundEvictor : public PeriodicWorker { tasks_.enqueue(std::make_pair(pid,cid)); } BackgroundEvictorStats getStats() const noexcept; + std::map getClassStats() const noexcept; private: // cache allocator's interface for evicting @@ -73,10 +74,12 @@ class BackgroundEvictor : public PeriodicWorker { // implements the actual logic of running the background evictor void work() override final; void checkAndRun(PoolId pid); - + + std::map evictions_per_class_; std::atomic numEvictedItems_{0}; std::atomic numEvictedItemsFromSchedule_{0}; std::atomic runCount_{0}; + std::atomic totalClasses_{0}; }; } // namespace cachelib } // namespace facebook diff --git a/cachelib/allocator/CacheAllocator.h b/cachelib/allocator/CacheAllocator.h index 4ae045b77d..d69a753f4a 100644 --- a/cachelib/allocator/CacheAllocator.h +++ b/cachelib/allocator/CacheAllocator.h @@ -1048,6 +1048,11 @@ class CacheAllocator : public CacheBase { auto stats = backgroundEvictor_ ? backgroundEvictor_->getStats() : BackgroundEvictorStats{}; return stats; } + + std::map getBackgroundEvictorClassStats() const { + auto stats = backgroundEvictor_ ? backgroundEvictor_->getClassStats() : std::map(); + return stats; + } // return the LruType of an item typename MMType::LruType getItemLruType(const Item& item) const; diff --git a/cachelib/allocator/CacheStats.h b/cachelib/allocator/CacheStats.h index dbda45ebae..1a290e2674 100644 --- a/cachelib/allocator/CacheStats.h +++ b/cachelib/allocator/CacheStats.h @@ -294,7 +294,11 @@ struct BackgroundEvictorStats { uint64_t numEvictedItemsFromSchedule{0}; // number of times we went executed the thread //TODO: is this def correct? - uint64_t numTraversals{0}; + uint64_t runCount{0}; + + uint64_t totalClasses{0}; + + }; diff --git a/cachelib/cachebench/cache/Cache-inl.h b/cachelib/cachebench/cache/Cache-inl.h index 75e207ded5..538bb2b73c 100644 --- a/cachelib/cachebench/cache/Cache-inl.h +++ b/cachelib/cachebench/cache/Cache-inl.h @@ -59,6 +59,11 @@ Cache::Cache(const CacheConfig& config, allocatorConfig_.enablePoolRebalancing( config_.getRebalanceStrategy(), std::chrono::seconds(config_.poolRebalanceIntervalSec)); + + //for another day + //allocatorConfig_.enablePoolOptimizer( + // config_.getPoolOptimizerStrategy(), + // std::chrono::seconds(config_.poolOptimizerIntervalSec)); allocatorConfig_.enableBackgroundEvictor( config_.getBackgroundEvictorStrategy(), @@ -529,11 +534,7 @@ Stats Cache::getStats() const { ret.numBackgroundEvictions = cacheStats.backgroundEvictorStats.numEvictedItems; ret.numBackgroundEvictionsFromSchedule = cacheStats.backgroundEvictorStats.numEvictedItemsFromSchedule; - ret.numBackgroundEvictorRuns = cacheStats.backgroundEvictorStats.numTraversals; - - ret.numCacheGets = cacheStats.numCacheGets; - ret.numCacheGetMiss = cacheStats.numCacheGetMiss; - ret.numRamDestructorCalls = cacheStats.numRamDestructorCalls; + ret.numBackgroundEvictorRuns = cacheStats.backgroundEvictorStats.runCount; ret.numNvmGets = cacheStats.numNvmGets; ret.numNvmGetMiss = cacheStats.numNvmGetMiss; ret.numNvmGetCoalesced = cacheStats.numNvmGetCoalesced; @@ -576,6 +577,8 @@ Stats Cache::getStats() const { ret.nvmCounters = cache_->getNvmCacheStatsMap(); } + ret.backgroundEvictionClasses = cache_->getBackgroundEvictorClassStats(); + // nvm stats from navy if (!isRamOnly() && !navyStats.empty()) { auto lookup = [&navyStats](const std::string& key) { diff --git a/cachelib/cachebench/cache/CacheStats.h b/cachelib/cachebench/cache/CacheStats.h index b81affc942..0de213f9ce 100644 --- a/cachelib/cachebench/cache/CacheStats.h +++ b/cachelib/cachebench/cache/CacheStats.h @@ -35,6 +35,7 @@ struct Stats { uint64_t numBackgroundEvictions{0}; uint64_t numBackgroundEvictionsFromSchedule{0}; uint64_t numBackgroundEvictorRuns{0}; + uint64_t numBackgroundEvictorClasses{0}; uint64_t numCacheGets{0}; uint64_t numCacheGetMiss{0}; @@ -103,6 +104,8 @@ struct Stats { // what to populate since not all of those are interesting when running // cachebench. std::unordered_map nvmCounters; + + std::map backgroundEvictionClasses; // errors from the nvm engine. std::unordered_map nvmErrors; @@ -124,6 +127,8 @@ struct Stats { out << folly::sformat("Background Tier 0 Evictions from schedule() : {:,}", numBackgroundEvictionsFromSchedule) << std::endl; out << folly::sformat("Background Tier 0 Eviction Runs : {:,}", numBackgroundEvictorRuns) << std::endl; + + out << folly::sformat("Background Tier 0 Total Classes Searched : {:,}", numBackgroundEvictorClasses) << std::endl; if (numCacheGets > 0) { out << folly::sformat("Cache Gets : {:,}", numCacheGets) << std::endl; @@ -282,6 +287,13 @@ struct Stats { out << it.first << " : " << it.second << std::endl; } } + + if (!backgroundEvictionClasses.empty()) { + out << "== Class Background Eviction Counters Map ==" << std::endl; + for (const auto& it : backgroundEvictionClasses) { + out << it.first << " : " << it.second << std::endl; + } + } if (numRamDestructorCalls > 0 || numNvmDestructorCalls > 0) { out << folly::sformat("Destructor executed from RAM {}, from NVM {}", From daf1793a55bca626ae7928839938a872310c93e0 Mon Sep 17 00:00:00 2001 From: Sounak Gupta Date: Thu, 14 Apr 2022 00:44:46 -0700 Subject: [PATCH 12/13] updated the background eviction stats to use AtomicCounters lib --- cachelib/allocator/BackgroundEvictor-inl.h | 24 ++++++++------- cachelib/allocator/BackgroundEvictor.h | 24 +++++++++++---- cachelib/allocator/CacheAllocator-inl.h | 2 +- cachelib/allocator/CacheAllocator.h | 4 +-- cachelib/allocator/CacheStats.h | 12 ++++---- cachelib/cachebench/cache/Cache-inl.h | 13 ++++++--- cachelib/cachebench/cache/CacheStats.h | 34 ++++++++++++++++------ 7 files changed, 77 insertions(+), 36 deletions(-) diff --git a/cachelib/allocator/BackgroundEvictor-inl.h b/cachelib/allocator/BackgroundEvictor-inl.h index 5908477479..24c93ad506 100644 --- a/cachelib/allocator/BackgroundEvictor-inl.h +++ b/cachelib/allocator/BackgroundEvictor-inl.h @@ -39,10 +39,11 @@ void BackgroundEvictor::work() { while (auto entry = tasks_.try_dequeue()) { auto [pid, cid] = entry.value(); auto batch = strategy_->calculateBatchSize(cache_, tid_, pid, cid); + stats.evictionSize.add(batch); auto evicted = BackgroundEvictorAPIWrapper::traverseAndEvictItems(cache_, tid_,pid,cid,batch); - numEvictedItemsFromSchedule_.fetch_add(1, std::memory_order_relaxed); - runCount_.fetch_add(1, std::memory_order_relaxed); + stats.numEvictedItemsFromSchedule.inc(); + stats.numTraversals.inc(); } } else { for (const auto pid : cache_.getRegularPoolIds()) { @@ -67,22 +68,25 @@ void BackgroundEvictor::checkAndRun(PoolId pid) { if (!batch) continue; + stats.evictionSize.add(batch); //try evicting BATCH items from the class in order to reach free target auto evicted = BackgroundEvictorAPIWrapper::traverseAndEvictItems(cache_, tid_,pid,cid,batch); - numEvictedItems_.fetch_add(evicted, std::memory_order_relaxed); + stats.numEvictedItems.add(evicted); } - runCount_.fetch_add(1, std::memory_order_relaxed); + stats.numTraversals.inc(); } template -BackgroundEvictorStats BackgroundEvictor::getStats() const noexcept { - BackgroundEvictorStats stats; - stats.numEvictedItems = numEvictedItems_.load(std::memory_order_relaxed); - stats.numTraversals = runCount_.load(std::memory_order_relaxed); - stats.numEvictedItemsFromSchedule = numEvictedItemsFromSchedule_.load(std::memory_order_relaxed); - return stats; +BackgroundEvictionStats BackgroundEvictor::getStats() const noexcept { + BackgroundEvictionStats evicStats; + evicStats.numEvictedItems = stats.numEvictedItems.get(); + evicStats.numEvictedItemsFromSchedule = stats.numEvictedItemsFromSchedule.get(); + evicStats.numTraversals = stats.numTraversals.get(); + evicStats.evictionSize = stats.evictionSize.get(); + + return evicStats; } } // namespace cachelib diff --git a/cachelib/allocator/BackgroundEvictor.h b/cachelib/allocator/BackgroundEvictor.h index bbcc11add0..3c21a254d9 100644 --- a/cachelib/allocator/BackgroundEvictor.h +++ b/cachelib/allocator/BackgroundEvictor.h @@ -22,6 +22,7 @@ #include "cachelib/allocator/CacheStats.h" #include "cachelib/common/PeriodicWorker.h" #include "cachelib/allocator/BackgroundEvictorStrategy.h" +#include "cachelib/common/AtomicCounter.h" namespace facebook { @@ -38,6 +39,20 @@ struct BackgroundEvictorAPIWrapper { } }; +struct BackgroundEvictorStats { + // items evicted + AtomicCounter numEvictedItems{0}; + + // items evicted from schedule + AtomicCounter numEvictedItemsFromSchedule; + + // traversals + AtomicCounter numTraversals{0}; + + // item eviction size + AtomicCounter evictionSize{0}; +}; + // Periodic worker that evicts items from tiers in batches // The primary aim is to reduce insertion times for new items in the // cache @@ -58,7 +73,8 @@ class BackgroundEvictor : public PeriodicWorker { void schedule(size_t pid, size_t cid) { tasks_.enqueue(std::make_pair(pid,cid)); } - BackgroundEvictorStats getStats() const noexcept; + + BackgroundEvictionStats getStats() const noexcept; private: // cache allocator's interface for evicting @@ -73,10 +89,8 @@ class BackgroundEvictor : public PeriodicWorker { // implements the actual logic of running the background evictor void work() override final; void checkAndRun(PoolId pid); - - std::atomic numEvictedItems_{0}; - std::atomic numEvictedItemsFromSchedule_{0}; - std::atomic runCount_{0}; + + BackgroundEvictorStats stats; }; } // namespace cachelib } // namespace facebook diff --git a/cachelib/allocator/CacheAllocator-inl.h b/cachelib/allocator/CacheAllocator-inl.h index 0761b0a7c6..297fc29f35 100644 --- a/cachelib/allocator/CacheAllocator-inl.h +++ b/cachelib/allocator/CacheAllocator-inl.h @@ -3676,7 +3676,7 @@ GlobalCacheStats CacheAllocator::getGlobalCacheStats() const { ret.nvmCacheEnabled = nvmCache_ ? nvmCache_->isEnabled() : false; ret.nvmUpTime = currTime - getNVMCacheCreationTime(); ret.reaperStats = getReaperStats(); - ret.backgroundEvictorStats = getBackgroundEvictorStats(); + ret.evictionStats = getBackgroundEvictorStats(); ret.numActiveHandles = getNumActiveHandles(); return ret; diff --git a/cachelib/allocator/CacheAllocator.h b/cachelib/allocator/CacheAllocator.h index 4ae045b77d..3885ecf3d4 100644 --- a/cachelib/allocator/CacheAllocator.h +++ b/cachelib/allocator/CacheAllocator.h @@ -1044,8 +1044,8 @@ class CacheAllocator : public CacheBase { } // returns the background evictor - BackgroundEvictorStats getBackgroundEvictorStats() const { - auto stats = backgroundEvictor_ ? backgroundEvictor_->getStats() : BackgroundEvictorStats{}; + BackgroundEvictionStats getBackgroundEvictorStats() const { + auto stats = backgroundEvictor_ ? backgroundEvictor_->getStats() : BackgroundEvictionStats{}; return stats; } diff --git a/cachelib/allocator/CacheStats.h b/cachelib/allocator/CacheStats.h index dbda45ebae..1289be2291 100644 --- a/cachelib/allocator/CacheStats.h +++ b/cachelib/allocator/CacheStats.h @@ -285,8 +285,8 @@ struct ReaperStats { uint64_t avgTraversalTimeMs{0}; }; -// Stats for background evictor -struct BackgroundEvictorStats { +// Eviction Stats +struct BackgroundEvictionStats { // the number of items this worker evicted by looking at pools/classes stats uint64_t numEvictedItems{0}; @@ -296,6 +296,8 @@ struct BackgroundEvictorStats { // number of times we went executed the thread //TODO: is this def correct? uint64_t numTraversals{0}; + // eviction size + uint64_t evictionSize{0}; }; // CacheMetadata type to export @@ -318,6 +320,9 @@ struct Stats; // Stats that apply globally in cache and // the ones that are aggregated over all pools struct GlobalCacheStats { + // background eviction stats + BackgroundEvictionStats evictionStats; + // number of calls to CacheAllocator::find uint64_t numCacheGets{0}; @@ -482,9 +487,6 @@ struct GlobalCacheStats { // stats related to the reaper ReaperStats reaperStats; - - // stats related to the background evictor - BackgroundEvictorStats backgroundEvictorStats; uint64_t numNvmRejectsByExpiry{}; uint64_t numNvmRejectsByClean{}; diff --git a/cachelib/cachebench/cache/Cache-inl.h b/cachelib/cachebench/cache/Cache-inl.h index 75e207ded5..87a743d568 100644 --- a/cachelib/cachebench/cache/Cache-inl.h +++ b/cachelib/cachebench/cache/Cache-inl.h @@ -522,14 +522,19 @@ Stats Cache::getStats() const { const auto navyStats = cache_->getNvmCacheStatsMap(); Stats ret; + ret.backgndEvicStats.nEvictedItems = + cacheStats.evictionStats.numEvictedItems; + ret.backgndEvicStats.nEvictedItemsFromSchedule = + cacheStats.evictionStats.numEvictedItemsFromSchedule; + ret.backgndEvicStats.nTraversals = + cacheStats.evictionStats.numTraversals; + ret.backgndEvicStats.evictionSize = + cacheStats.evictionStats.evictionSize; + ret.numEvictions = aggregate.numEvictions(); ret.numItems = aggregate.numItems(); ret.allocAttempts = cacheStats.allocAttempts; ret.allocFailures = cacheStats.allocFailures; - - ret.numBackgroundEvictions = cacheStats.backgroundEvictorStats.numEvictedItems; - ret.numBackgroundEvictionsFromSchedule = cacheStats.backgroundEvictorStats.numEvictedItemsFromSchedule; - ret.numBackgroundEvictorRuns = cacheStats.backgroundEvictorStats.numTraversals; ret.numCacheGets = cacheStats.numCacheGets; ret.numCacheGetMiss = cacheStats.numCacheGetMiss; diff --git a/cachelib/cachebench/cache/CacheStats.h b/cachelib/cachebench/cache/CacheStats.h index b81affc942..8a6ce0cdb3 100644 --- a/cachelib/cachebench/cache/CacheStats.h +++ b/cachelib/cachebench/cache/CacheStats.h @@ -25,17 +25,30 @@ DECLARE_bool(report_api_latency); namespace facebook { namespace cachelib { namespace cachebench { + +struct BackgroundEvictionStats { + // the number of items this worker evicted by looking at pools/classes stats + uint64_t nEvictedItems{0}; + + // the number of items this worker evicted for pools/classes requested by schedule call + uint64_t nEvictedItemsFromSchedule{0}; + + // number of times we went executed the thread //TODO: is this def correct? + uint64_t nTraversals{0}; + + // size of evicted items + uint64_t evictionSize; +}; + struct Stats { + BackgroundEvictionStats backgndEvicStats; + uint64_t numEvictions{0}; uint64_t numItems{0}; uint64_t allocAttempts{0}; uint64_t allocFailures{0}; - uint64_t numBackgroundEvictions{0}; - uint64_t numBackgroundEvictionsFromSchedule{0}; - uint64_t numBackgroundEvictorRuns{0}; - uint64_t numCacheGets{0}; uint64_t numCacheGetMiss{0}; uint64_t numRamDestructorCalls{0}; @@ -118,12 +131,15 @@ struct Stats { invertPctFn(allocFailures, allocAttempts)) << std::endl; out << folly::sformat("RAM Evictions : {:,}", numEvictions) << std::endl; - - out << folly::sformat("Background Tier 0 Evictions : {:,}", numBackgroundEvictions) << std::endl; - out << folly::sformat("Background Tier 0 Evictions from schedule() : {:,}", numBackgroundEvictionsFromSchedule) << std::endl; - - out << folly::sformat("Background Tier 0 Eviction Runs : {:,}", numBackgroundEvictorRuns) << std::endl; + out << folly::sformat("Tier 0 Background Evicted items : {:,}", + backgndEvicStats.nEvictedItems) << std::endl; + out << folly::sformat("Tier 0 Background Evicted items from schedule : {:,}", + backgndEvicStats.nEvictedItemsFromSchedule) << std::endl; + out << folly::sformat("Tier 0 Background Traversals : {:,}", + backgndEvicStats.nTraversals) << std::endl; + out << folly::sformat("Tier 0 Background Evicted Size : {:,}", + backgndEvicStats.evictionSize) << std::endl; if (numCacheGets > 0) { out << folly::sformat("Cache Gets : {:,}", numCacheGets) << std::endl; From 6006cc851a531197a6b3aefb7eaa3642437f4cf1 Mon Sep 17 00:00:00 2001 From: Daniel Byrne Date: Thu, 14 Apr 2022 09:17:18 -0400 Subject: [PATCH 13/13] updated stats for alloc size --- cachelib/allocator/BackgroundEvictor-inl.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cachelib/allocator/BackgroundEvictor-inl.h b/cachelib/allocator/BackgroundEvictor-inl.h index 8f246bcb63..a0f8c9b299 100644 --- a/cachelib/allocator/BackgroundEvictor-inl.h +++ b/cachelib/allocator/BackgroundEvictor-inl.h @@ -65,9 +65,9 @@ void BackgroundEvictor::checkAndRun(PoolId pid) { unsigned int evictions = 0; unsigned int classes = 0; for (auto& cid : mpStats.classIds) { + classes++; auto batch = strategy_->calculateBatchSize(cache_,tid_,pid,cid); if (!batch) { - classes++; continue; } @@ -76,7 +76,7 @@ void BackgroundEvictor::checkAndRun(PoolId pid) { BackgroundEvictorAPIWrapper::traverseAndEvictItems(cache_, tid_,pid,cid,batch); evictions += evicted; - const size_t cid_id = (size_t)cid; + const size_t cid_id = (size_t)mpStats.acStats.at(cid).allocSize; auto it = evictions_per_class_.find(cid_id); if (it != evictions_per_class_.end()) { it->second += evicted;