jemalloc_nodump_allocator.cc 11 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304
  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. #include "memory/jemalloc_nodump_allocator.h"
  6. #include <string>
  7. #include <thread>
  8. #include "port/likely.h"
  9. #include "port/port.h"
  10. #include "rocksdb/convenience.h"
  11. #include "rocksdb/utilities/customizable_util.h"
  12. #include "rocksdb/utilities/object_registry.h"
  13. #include "rocksdb/utilities/options_type.h"
  14. #include "util/fastrange.h"
  15. #include "util/random.h"
  16. #include "util/string_util.h"
  17. namespace ROCKSDB_NAMESPACE {
  18. #ifdef ROCKSDB_JEMALLOC_NODUMP_ALLOCATOR
  19. std::atomic<extent_alloc_t*> JemallocNodumpAllocator::original_alloc_{nullptr};
  20. #endif // ROCKSDB_JEMALLOC_NODUMP_ALLOCATOR
  21. static std::unordered_map<std::string, OptionTypeInfo> jemalloc_type_info = {
  22. {"limit_tcache_size",
  23. {offsetof(struct JemallocAllocatorOptions, limit_tcache_size),
  24. OptionType::kBoolean, OptionVerificationType::kNormal,
  25. OptionTypeFlags::kNone}},
  26. {"tcache_size_lower_bound",
  27. {offsetof(struct JemallocAllocatorOptions, tcache_size_lower_bound),
  28. OptionType::kSizeT, OptionVerificationType::kNormal,
  29. OptionTypeFlags::kNone}},
  30. {"tcache_size_upper_bound",
  31. {offsetof(struct JemallocAllocatorOptions, tcache_size_upper_bound),
  32. OptionType::kSizeT, OptionVerificationType::kNormal,
  33. OptionTypeFlags::kNone}},
  34. {"num_arenas",
  35. {offsetof(struct JemallocAllocatorOptions, num_arenas), OptionType::kSizeT,
  36. OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
  37. };
  38. bool JemallocNodumpAllocator::IsSupported(std::string* why) {
  39. #ifndef ROCKSDB_JEMALLOC
  40. *why = "Not compiled with ROCKSDB_JEMALLOC";
  41. return false;
  42. #else
  43. static const std::string unsupported =
  44. "JemallocNodumpAllocator only available with jemalloc version >= 5 "
  45. "and MADV_DONTDUMP is available.";
  46. if (!HasJemalloc()) {
  47. *why = unsupported;
  48. return false;
  49. }
  50. #ifndef ROCKSDB_JEMALLOC_NODUMP_ALLOCATOR
  51. *why = unsupported;
  52. return false;
  53. #else
  54. return true;
  55. #endif // ROCKSDB_JEMALLOC_NODUMP_ALLOCATOR
  56. #endif // ROCKSDB_MALLOC
  57. }
  58. JemallocNodumpAllocator::JemallocNodumpAllocator(
  59. const JemallocAllocatorOptions& options)
  60. : options_(options)
  61. #ifdef ROCKSDB_JEMALLOC_NODUMP_ALLOCATOR
  62. ,
  63. tcache_(&JemallocNodumpAllocator::DestroyThreadSpecificCache) {
  64. #else // ROCKSDB_JEMALLOC_NODUMP_ALLOCATOR
  65. {
  66. #endif // ROCKSDB_JEMALLOC_NODUMP_ALLOCATOR
  67. RegisterOptions(&options_, &jemalloc_type_info);
  68. }
  69. #ifdef ROCKSDB_JEMALLOC_NODUMP_ALLOCATOR
  70. JemallocNodumpAllocator::~JemallocNodumpAllocator() {
  71. // Destroy tcache before destroying arena.
  72. autovector<void*> tcache_list;
  73. tcache_.Scrape(&tcache_list, nullptr);
  74. for (void* tcache_index : tcache_list) {
  75. DestroyThreadSpecificCache(tcache_index);
  76. }
  77. for (auto arena_index : arena_indexes_) {
  78. // Destroy arena. Silently ignore error.
  79. Status s = DestroyArena(arena_index);
  80. assert(s.ok());
  81. s.PermitUncheckedError();
  82. }
  83. }
  84. size_t JemallocNodumpAllocator::UsableSize(void* p,
  85. size_t /*allocation_size*/) const {
  86. return malloc_usable_size(static_cast<void*>(p));
  87. }
  88. void* JemallocNodumpAllocator::Allocate(size_t size) {
  89. int tcache_flag = GetThreadSpecificCache(size);
  90. uint32_t arena_index = GetArenaIndex();
  91. return mallocx(size, MALLOCX_ARENA(arena_index) | tcache_flag);
  92. }
  93. void JemallocNodumpAllocator::Deallocate(void* p) {
  94. // Obtain tcache.
  95. size_t size = 0;
  96. if (options_.limit_tcache_size) {
  97. size = malloc_usable_size(p);
  98. }
  99. int tcache_flag = GetThreadSpecificCache(size);
  100. // No need to pass arena index to dallocx(). Jemalloc will find arena index
  101. // from its own metadata.
  102. dallocx(p, tcache_flag);
  103. }
  104. uint32_t JemallocNodumpAllocator::GetArenaIndex() const {
  105. if (arena_indexes_.size() == 1) {
  106. return arena_indexes_[0];
  107. }
  108. static std::atomic<uint32_t> next_seed = 0;
  109. // Core-local may work in place of `thread_local` as we should be able to
  110. // tolerate occasional stale reads in thread migration cases. However we need
  111. // to make Random thread-safe and prevent cacheline bouncing. Whether this is
  112. // worthwhile is still an open question.
  113. thread_local Random tl_random(next_seed.fetch_add(1));
  114. return arena_indexes_[FastRange32(
  115. tl_random.Next(), static_cast<uint32_t>(arena_indexes_.size()))];
  116. }
  117. Status JemallocNodumpAllocator::InitializeArenas() {
  118. assert(!init_);
  119. init_ = true;
  120. for (size_t i = 0; i < options_.num_arenas; i++) {
  121. // Create arena.
  122. unsigned arena_index;
  123. size_t arena_index_size = sizeof(arena_index);
  124. int ret =
  125. mallctl("arenas.create", &arena_index, &arena_index_size, nullptr, 0);
  126. if (ret != 0) {
  127. return Status::Incomplete(
  128. "Failed to create jemalloc arena, error code: " +
  129. std::to_string(ret));
  130. }
  131. arena_indexes_.push_back(arena_index);
  132. // Read existing hooks.
  133. std::string key =
  134. "arena." + std::to_string(arena_indexes_[i]) + ".extent_hooks";
  135. extent_hooks_t* hooks;
  136. size_t hooks_size = sizeof(hooks);
  137. ret = mallctl(key.c_str(), &hooks, &hooks_size, nullptr, 0);
  138. if (ret != 0) {
  139. return Status::Incomplete("Failed to read existing hooks, error code: " +
  140. std::to_string(ret));
  141. }
  142. // Store existing alloc.
  143. extent_alloc_t* original_alloc = hooks->alloc;
  144. extent_alloc_t* expected = nullptr;
  145. bool success =
  146. JemallocNodumpAllocator::original_alloc_.compare_exchange_strong(
  147. expected, original_alloc);
  148. if (!success && original_alloc != expected) {
  149. // This could happen if jemalloc creates new arenas with different initial
  150. // values in their `alloc` function pointers. See `original_alloc_` API
  151. // doc for more details.
  152. return Status::Incomplete("Original alloc conflict.");
  153. }
  154. // Set the custom hook.
  155. per_arena_hooks_.emplace_back();
  156. per_arena_hooks_.back().reset(new extent_hooks_t(*hooks));
  157. per_arena_hooks_.back()->alloc = &JemallocNodumpAllocator::Alloc;
  158. extent_hooks_t* hooks_ptr = per_arena_hooks_.back().get();
  159. ret = mallctl(key.c_str(), nullptr, nullptr, &hooks_ptr, sizeof(hooks_ptr));
  160. if (ret != 0) {
  161. return Status::Incomplete("Failed to set custom hook, error code: " +
  162. std::to_string(ret));
  163. }
  164. }
  165. return Status::OK();
  166. }
  167. #endif // ROCKSDB_JEMALLOC_NODUMP_ALLOCATOR
  168. Status JemallocNodumpAllocator::PrepareOptions(
  169. const ConfigOptions& config_options) {
  170. std::string message;
  171. if (!IsSupported(&message)) {
  172. return Status::NotSupported(message);
  173. } else if (options_.limit_tcache_size &&
  174. options_.tcache_size_lower_bound >=
  175. options_.tcache_size_upper_bound) {
  176. return Status::InvalidArgument(
  177. "tcache_size_lower_bound larger or equal to tcache_size_upper_bound.");
  178. } else if (options_.num_arenas < 1) {
  179. return Status::InvalidArgument("num_arenas must be a positive integer");
  180. } else if (IsMutable()) {
  181. Status s = MemoryAllocator::PrepareOptions(config_options);
  182. #ifdef ROCKSDB_JEMALLOC_NODUMP_ALLOCATOR
  183. if (s.ok()) {
  184. s = InitializeArenas();
  185. }
  186. #endif // ROCKSDB_JEMALLOC_NODUMP_ALLOCATOR
  187. return s;
  188. } else {
  189. // Already prepared
  190. return Status::OK();
  191. }
  192. }
  193. #ifdef ROCKSDB_JEMALLOC_NODUMP_ALLOCATOR
  194. int JemallocNodumpAllocator::GetThreadSpecificCache(size_t size) {
  195. // We always enable tcache. The only corner case is when there are a ton of
  196. // threads accessing with low frequency, then it could consume a lot of
  197. // memory (may reach # threads * ~1MB) without bringing too much benefit.
  198. if (options_.limit_tcache_size && (size <= options_.tcache_size_lower_bound ||
  199. size > options_.tcache_size_upper_bound)) {
  200. return MALLOCX_TCACHE_NONE;
  201. }
  202. unsigned* tcache_index = static_cast<unsigned*>(tcache_.Get());
  203. if (UNLIKELY(tcache_index == nullptr)) {
  204. // Instantiate tcache.
  205. tcache_index = new unsigned(0);
  206. size_t tcache_index_size = sizeof(unsigned);
  207. int ret =
  208. mallctl("tcache.create", tcache_index, &tcache_index_size, nullptr, 0);
  209. if (ret != 0) {
  210. // No good way to expose the error. Silently disable tcache.
  211. delete tcache_index;
  212. return MALLOCX_TCACHE_NONE;
  213. }
  214. tcache_.Reset(static_cast<void*>(tcache_index));
  215. }
  216. return MALLOCX_TCACHE(*tcache_index);
  217. }
  218. void* JemallocNodumpAllocator::Alloc(extent_hooks_t* extent, void* new_addr,
  219. size_t size, size_t alignment, bool* zero,
  220. bool* commit, unsigned arena_ind) {
  221. extent_alloc_t* original_alloc =
  222. original_alloc_.load(std::memory_order_relaxed);
  223. assert(original_alloc != nullptr);
  224. void* result = original_alloc(extent, new_addr, size, alignment, zero, commit,
  225. arena_ind);
  226. if (result != nullptr) {
  227. int ret = madvise(result, size, MADV_DONTDUMP);
  228. if (ret != 0) {
  229. fprintf(
  230. stderr,
  231. "JemallocNodumpAllocator failed to set MADV_DONTDUMP, error code: %d",
  232. ret);
  233. assert(false);
  234. }
  235. }
  236. return result;
  237. }
  238. Status JemallocNodumpAllocator::DestroyArena(uint32_t arena_index) {
  239. assert(arena_index != 0);
  240. std::string key = "arena." + std::to_string(arena_index) + ".destroy";
  241. int ret = mallctl(key.c_str(), nullptr, 0, nullptr, 0);
  242. if (ret != 0) {
  243. return Status::Incomplete("Failed to destroy jemalloc arena, error code: " +
  244. std::to_string(ret));
  245. }
  246. return Status::OK();
  247. }
  248. void JemallocNodumpAllocator::DestroyThreadSpecificCache(void* ptr) {
  249. assert(ptr != nullptr);
  250. unsigned* tcache_index = static_cast<unsigned*>(ptr);
  251. size_t tcache_index_size = sizeof(unsigned);
  252. int ret __attribute__((__unused__)) =
  253. mallctl("tcache.destroy", nullptr, 0, tcache_index, tcache_index_size);
  254. // Silently ignore error.
  255. assert(ret == 0);
  256. delete tcache_index;
  257. }
  258. #endif // ROCKSDB_JEMALLOC_NODUMP_ALLOCATOR
  259. Status NewJemallocNodumpAllocator(
  260. const JemallocAllocatorOptions& options,
  261. std::shared_ptr<MemoryAllocator>* memory_allocator) {
  262. if (memory_allocator == nullptr) {
  263. return Status::InvalidArgument("memory_allocator must be non-null.");
  264. }
  265. #ifndef ROCKSDB_JEMALLOC
  266. (void)options;
  267. return Status::NotSupported("Not compiled with JEMALLOC");
  268. #else
  269. std::unique_ptr<MemoryAllocator> allocator(
  270. new JemallocNodumpAllocator(options));
  271. Status s = allocator->PrepareOptions(ConfigOptions());
  272. if (s.ok()) {
  273. memory_allocator->reset(allocator.release());
  274. }
  275. return s;
  276. #endif
  277. }
  278. } // namespace ROCKSDB_NAMESPACE