1 // Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
2 //  This source code is licensed under both the GPLv2 (found in the
3 //  COPYING file in the root directory) and Apache 2.0 License
4 //  (found in the LICENSE.Apache file in the root directory).
5 //
6 // Copyright (c) 2011 The LevelDB Authors. All rights reserved.
7 // Use of this source code is governed by a BSD-style license that can be
8 // found in the LICENSE file. See the AUTHORS file for names of contributors.
9 //
10 // A Cache is an interface that maps keys to values.  It has internal
11 // synchronization and may be safely accessed concurrently from
12 // multiple threads.  It may automatically evict entries to make room
13 // for new entries.  Values have a specified charge against the cache
14 // capacity.  For example, a cache where the values are variable
15 // length strings, may use the length of the string as the charge for
16 // the string.
17 //
18 // A builtin cache implementation with a least-recently-used eviction
19 // policy is provided.  Clients may use their own implementations if
20 // they want something more sophisticated (like scan-resistance, a
21 // custom eviction policy, variable cache sizing, etc.)
22 
23 #pragma once
24 
25 #include <stdint.h>
26 #include <memory>
27 #include <string>
28 #include "rocksdb/memory_allocator.h"
29 #include "rocksdb/slice.h"
30 #include "rocksdb/statistics.h"
31 #include "rocksdb/status.h"
32 
33 namespace ROCKSDB_NAMESPACE {
34 
35 class Cache;
36 
37 extern const bool kDefaultToAdaptiveMutex;
38 
39 enum CacheMetadataChargePolicy {
40   kDontChargeCacheMetadata,
41   kFullChargeCacheMetadata
42 };
43 const CacheMetadataChargePolicy kDefaultCacheMetadataChargePolicy =
44     kFullChargeCacheMetadata;
45 
46 struct LRUCacheOptions {
47   // Capacity of the cache.
48   size_t capacity = 0;
49 
50   // Cache is sharded into 2^num_shard_bits shards,
51   // by hash of key. Refer to NewLRUCache for further
52   // information.
53   int num_shard_bits = -1;
54 
55   // If strict_capacity_limit is set,
56   // insert to the cache will fail when cache is full.
57   bool strict_capacity_limit = false;
58 
59   // Percentage of cache reserved for high priority entries.
60   // If greater than zero, the LRU list will be split into a high-pri
61   // list and a low-pri list. High-pri entries will be insert to the
62   // tail of high-pri list, while low-pri entries will be first inserted to
63   // the low-pri list (the midpoint). This is refered to as
64   // midpoint insertion strategy to make entries never get hit in cache
65   // age out faster.
66   //
67   // See also
68   // BlockBasedTableOptions::cache_index_and_filter_blocks_with_high_priority.
69   double high_pri_pool_ratio = 0.5;
70 
71   // If non-nullptr will use this allocator instead of system allocator when
72   // allocating memory for cache blocks. Call this method before you start using
73   // the cache!
74   //
75   // Caveat: when the cache is used as block cache, the memory allocator is
76   // ignored when dealing with compression libraries that allocate memory
77   // internally (currently only XPRESS).
78   std::shared_ptr<MemoryAllocator> memory_allocator;
79 
80   // Whether to use adaptive mutexes for cache shards. Note that adaptive
81   // mutexes need to be supported by the platform in order for this to have any
82   // effect. The default value is true if RocksDB is compiled with
83   // -DROCKSDB_DEFAULT_TO_ADAPTIVE_MUTEX, false otherwise.
84   bool use_adaptive_mutex = kDefaultToAdaptiveMutex;
85 
86   CacheMetadataChargePolicy metadata_charge_policy =
87       kDefaultCacheMetadataChargePolicy;
88 
LRUCacheOptionsLRUCacheOptions89   LRUCacheOptions() {}
90   LRUCacheOptions(size_t _capacity, int _num_shard_bits,
91                   bool _strict_capacity_limit, double _high_pri_pool_ratio,
92                   std::shared_ptr<MemoryAllocator> _memory_allocator = nullptr,
93                   bool _use_adaptive_mutex = kDefaultToAdaptiveMutex,
94                   CacheMetadataChargePolicy _metadata_charge_policy =
95                       kDefaultCacheMetadataChargePolicy)
capacityLRUCacheOptions96       : capacity(_capacity),
97         num_shard_bits(_num_shard_bits),
98         strict_capacity_limit(_strict_capacity_limit),
99         high_pri_pool_ratio(_high_pri_pool_ratio),
100         memory_allocator(std::move(_memory_allocator)),
101         use_adaptive_mutex(_use_adaptive_mutex),
102         metadata_charge_policy(_metadata_charge_policy) {}
103 };
104 
105 // Create a new cache with a fixed size capacity. The cache is sharded
106 // to 2^num_shard_bits shards, by hash of the key. The total capacity
107 // is divided and evenly assigned to each shard. If strict_capacity_limit
108 // is set, insert to the cache will fail when cache is full. User can also
109 // set percentage of the cache reserves for high priority entries via
110 // high_pri_pool_pct.
111 // num_shard_bits = -1 means it is automatically determined: every shard
112 // will be at least 512KB and number of shard bits will not exceed 6.
113 extern std::shared_ptr<Cache> NewLRUCache(
114     size_t capacity, int num_shard_bits = -1,
115     bool strict_capacity_limit = false, double high_pri_pool_ratio = 0.5,
116     std::shared_ptr<MemoryAllocator> memory_allocator = nullptr,
117     bool use_adaptive_mutex = kDefaultToAdaptiveMutex,
118     CacheMetadataChargePolicy metadata_charge_policy =
119         kDefaultCacheMetadataChargePolicy);
120 
121 extern std::shared_ptr<Cache> NewLRUCache(const LRUCacheOptions& cache_opts);
122 
123 // Similar to NewLRUCache, but create a cache based on CLOCK algorithm with
124 // better concurrent performance in some cases. See util/clock_cache.cc for
125 // more detail.
126 //
127 // Return nullptr if it is not supported.
128 extern std::shared_ptr<Cache> NewClockCache(
129     size_t capacity, int num_shard_bits = -1,
130     bool strict_capacity_limit = false,
131     CacheMetadataChargePolicy metadata_charge_policy =
132         kDefaultCacheMetadataChargePolicy);
133 class Cache {
134  public:
135   class Deleter {
136    public:
137     virtual ~Deleter() = default;
138 
139     virtual void operator()(const Slice& key, void* value) = 0;
140   };
141 
142   // Depending on implementation, cache entries with high priority could be less
143   // likely to get evicted than low priority entries.
144   enum class Priority { HIGH, LOW };
145 
146   Cache(std::shared_ptr<MemoryAllocator> allocator = nullptr)
memory_allocator_(std::move (allocator))147       : memory_allocator_(std::move(allocator)) {}
148   // No copying allowed
149   Cache(const Cache&) = delete;
150   Cache& operator=(const Cache&) = delete;
151 
152   // Destroys all existing entries by calling the "deleter"
153   // function that was passed via the Insert() function.
154   //
155   // @See Insert
~Cache()156   virtual ~Cache() {}
157 
158   // Opaque handle to an entry stored in the cache.
159   struct Handle {};
160 
161   // The type of the Cache
162   virtual const char* Name() const = 0;
163 
164   // Insert a mapping from key->value into the cache and assign it
165   // the specified charge against the total cache capacity.
166   // If strict_capacity_limit is true and cache reaches its full capacity,
167   // return Status::Incomplete.
168   //
169   // If handle is not nullptr, returns a handle that corresponds to the
170   // mapping. The caller must call this->Release(handle) when the returned
171   // mapping is no longer needed. In case of error caller is responsible to
172   // cleanup the value (i.e. calling "deleter").
173   //
174   // If handle is nullptr, it is as if Release is called immediately after
175   // insert. In case of error value will be cleanup.
176   //
177   // When the inserted entry is no longer needed, the key and
178   // value will be passed to "deleter". It is the caller's responsibility to
179   // ensure that the deleter outlives the cache entries referring to it.
180   virtual Status Insert(const Slice& key, void* value, size_t charge,
181                         Deleter* deleter, Handle** handle = nullptr,
182                         Priority priority = Priority::LOW) = 0;
183 
184   // If the cache has no mapping for "key", returns nullptr.
185   //
186   // Else return a handle that corresponds to the mapping.  The caller
187   // must call this->Release(handle) when the returned mapping is no
188   // longer needed.
189   // If stats is not nullptr, relative tickers could be used inside the
190   // function.
191   virtual Handle* Lookup(const Slice& key, Statistics* stats = nullptr) = 0;
192 
193   // Increments the reference count for the handle if it refers to an entry in
194   // the cache. Returns true if refcount was incremented; otherwise, returns
195   // false.
196   // REQUIRES: handle must have been returned by a method on *this.
197   virtual bool Ref(Handle* handle) = 0;
198 
199   /**
200    * Release a mapping returned by a previous Lookup(). A released entry might
201    * still  remain in cache in case it is later looked up by others. If
202    * force_erase is set then it also erase it from the cache if there is no
203    * other reference to  it. Erasing it should call the deleter function that
204    * was provided when the
205    * entry was inserted.
206    *
207    * Returns true if the entry was also erased.
208    */
209   // REQUIRES: handle must not have been released yet.
210   // REQUIRES: handle must have been returned by a method on *this.
211   virtual bool Release(Handle* handle, bool force_erase = false) = 0;
212 
213   // Return the value encapsulated in a handle returned by a
214   // successful Lookup().
215   // REQUIRES: handle must not have been released yet.
216   // REQUIRES: handle must have been returned by a method on *this.
217   virtual void* Value(Handle* handle) = 0;
218 
219   // If the cache contains entry for key, erase it.  Note that the
220   // underlying entry will be kept around until all existing handles
221   // to it have been released.
222   virtual void Erase(const Slice& key) = 0;
223   // Return a new numeric id.  May be used by multiple clients who are
224   // sharding the same cache to partition the key space.  Typically the
225   // client will allocate a new id at startup and prepend the id to
226   // its cache keys.
227   virtual uint64_t NewId() = 0;
228 
229   // sets the maximum configured capacity of the cache. When the new
230   // capacity is less than the old capacity and the existing usage is
231   // greater than new capacity, the implementation will do its best job to
232   // purge the released entries from the cache in order to lower the usage
233   virtual void SetCapacity(size_t capacity) = 0;
234 
235   // Set whether to return error on insertion when cache reaches its full
236   // capacity.
237   virtual void SetStrictCapacityLimit(bool strict_capacity_limit) = 0;
238 
239   // Get the flag whether to return error on insertion when cache reaches its
240   // full capacity.
241   virtual bool HasStrictCapacityLimit() const = 0;
242 
243   // returns the maximum configured capacity of the cache
244   virtual size_t GetCapacity() const = 0;
245 
246   // returns the memory size for the entries residing in the cache.
247   virtual size_t GetUsage() const = 0;
248 
249   // returns the memory size for a specific entry in the cache.
250   virtual size_t GetUsage(Handle* handle) const = 0;
251 
252   // returns the memory size for the entries in use by the system
253   virtual size_t GetPinnedUsage() const = 0;
254 
255   // returns the charge for the specific entry in the cache.
256   virtual size_t GetCharge(Handle* handle) const = 0;
257 
258   // Call this on shutdown if you want to speed it up. Cache will disown
259   // any underlying data and will not free it on delete. This call will leak
260   // memory - call this only if you're shutting down the process.
261   // Any attempts of using cache after this call will fail terribly.
262   // Always delete the DB object before calling this method!
DisownData()263   virtual void DisownData(){
264       // default implementation is noop
265   }
266 
267   // Apply callback to all entries in the cache
268   // If thread_safe is true, it will also lock the accesses. Otherwise, it will
269   // access the cache without the lock held
270   virtual void ApplyToAllCacheEntries(void (*callback)(void*, size_t),
271                                       bool thread_safe) = 0;
272 
273   // Remove all entries.
274   // Prerequisite: no entry is referenced.
275   virtual void EraseUnRefEntries() = 0;
276 
GetPrintableOptions()277   virtual std::string GetPrintableOptions() const { return ""; }
278 
memory_allocator()279   MemoryAllocator* memory_allocator() const { return memory_allocator_.get(); }
280 
281  private:
282   std::shared_ptr<MemoryAllocator> memory_allocator_;
283 };
284 
285 }  // namespace ROCKSDB_NAMESPACE
286