move StaticMeta::onThreadExit into libfolly_thread_local.so
authorEric Niebler <eniebler@fb.com>
Wed, 16 Mar 2016 16:54:38 +0000 (09:54 -0700)
committerFacebook Github Bot 1 <facebook-github-bot-1-bot@fb.com>
Wed, 16 Mar 2016 17:05:27 +0000 (10:05 -0700)
Summary: Fix folly/test:thread_local_test in the shared library scenario by moving onThreadExit into libfolly_thread_local.so, avoiding the crash when a user's .so is dlclosed.

Reviewed By: andriigrynenko

Differential Revision: D2919287

fb-gh-sync-id: ecc4220fd40203289366eb1a6391b8b6d971e65f
shipit-source-id: ecc4220fd40203289366eb1a6391b8b6d971e65f

folly/ThreadLocal.h
folly/detail/ThreadLocalDetail.cpp
folly/detail/ThreadLocalDetail.h

index 8f98ead62ffc1cc402bab5ba42720dc4ebc9263c..30b8042bb337d29aebd111ea4945159b973d8e36 100644 (file)
@@ -162,7 +162,7 @@ class ThreadLocalPtr {
   }
 
   T* get() const {
-    threadlocal_detail::ElementWrapper& w = StaticMeta::get(&id_);
+    threadlocal_detail::ElementWrapper& w = StaticMeta::instance().get(&id_);
     return static_cast<T*>(w.ptr);
   }
 
@@ -175,13 +175,13 @@ class ThreadLocalPtr {
   }
 
   T* release() {
-    threadlocal_detail::ElementWrapper& w = StaticMeta::get(&id_);
+    threadlocal_detail::ElementWrapper& w = StaticMeta::instance().get(&id_);
 
     return static_cast<T*>(w.release());
   }
 
   void reset(T* newPtr = nullptr) {
-    threadlocal_detail::ElementWrapper& w = StaticMeta::get(&id_);
+    threadlocal_detail::ElementWrapper& w = StaticMeta::instance().get(&id_);
 
     if (w.ptr != newPtr) {
       w.dispose(TLPDestructionMode::THIS_THREAD);
@@ -202,7 +202,7 @@ class ThreadLocalPtr {
    */
   template <class Deleter>
   void reset(T* newPtr, Deleter deleter) {
-    threadlocal_detail::ElementWrapper& w = StaticMeta::get(&id_);
+    threadlocal_detail::ElementWrapper& w = StaticMeta::instance().get(&id_);
     if (w.ptr != newPtr) {
       w.dispose(TLPDestructionMode::THIS_THREAD);
       w.set(newPtr, deleter);
@@ -215,7 +215,7 @@ class ThreadLocalPtr {
   class Accessor {
     friend class ThreadLocalPtr<T,Tag>;
 
-    threadlocal_detail::StaticMeta<Tag>& meta_;
+    threadlocal_detail::StaticMetaBase& meta_;
     std::mutex* lock_;
     uint32_t id_;
 
@@ -230,7 +230,7 @@ class ThreadLocalPtr {
           boost::bidirectional_traversal_tag> {   // traversal
       friend class Accessor;
       friend class boost::iterator_core_access;
-      const Accessor* const accessor_;
+      const Accessor* accessor_;
       threadlocal_detail::ThreadEntry* e_;
 
       void increment() {
@@ -337,12 +337,12 @@ class ThreadLocalPtr {
   Accessor accessAllThreads() const {
     static_assert(!std::is_same<Tag, void>::value,
                   "Must use a unique Tag to use the accessAllThreads feature");
-    return Accessor(id_.getOrAllocate());
+    return Accessor(id_.getOrAllocate(StaticMeta::instance()));
   }
 
  private:
   void destroy() {
-    StaticMeta::destroy(&id_);
+    StaticMeta::instance().destroy(&id_);
   }
 
   // non-copyable
index 7ea0a43d8f3edc4faf21b19706cf8613ef5a6ccc..afe7c58415c3d943d330d35a29c99b06ba6f9498 100644 (file)
 
 namespace folly { namespace threadlocal_detail {
 
+StaticMetaBase::StaticMetaBase(ThreadEntry* (*threadEntry)())
+    : nextId_(1), threadEntry_(threadEntry) {
+  head_.next = head_.prev = &head_;
+  int ret = pthread_key_create(&pthreadKey_, &onThreadExit);
+  checkPosixError(ret, "pthread_key_create failed");
+  PthreadKeyUnregister::registerKey(pthreadKey_);
+}
+
+void StaticMetaBase::onThreadExit(void* ptr) {
+  std::unique_ptr<ThreadEntry> threadEntry(static_cast<ThreadEntry*>(ptr));
+  DCHECK_GT(threadEntry->elementsCapacity, 0);
+  auto& meta = *threadEntry->meta;
+  {
+    std::lock_guard<std::mutex> g(meta.lock_);
+    meta.erase(threadEntry.get());
+    // No need to hold the lock any longer; the ThreadEntry is private to this
+    // thread now that it's been removed from meta.
+  }
+  // NOTE: User-provided deleter / object dtor itself may be using ThreadLocal
+  // with the same Tag, so dispose() calls below may (re)create some of the
+  // elements or even increase elementsCapacity, thus multiple cleanup rounds
+  // may be required.
+  for (bool shouldRun = true; shouldRun;) {
+    shouldRun = false;
+    FOR_EACH_RANGE (i, 0, threadEntry->elementsCapacity) {
+      if (threadEntry->elements[i].dispose(TLPDestructionMode::THIS_THREAD)) {
+        shouldRun = true;
+      }
+    }
+  }
+  free(threadEntry->elements);
+  threadEntry->elements = nullptr;
+  threadEntry->meta = nullptr;
+}
+
+uint32_t StaticMetaBase::allocate(EntryID* ent) {
+  uint32_t id;
+  auto& meta = *this;
+  std::lock_guard<std::mutex> g(meta.lock_);
+
+  id = ent->value.load();
+  if (id != kEntryIDInvalid) {
+    return id;
+  }
+
+  if (!meta.freeIds_.empty()) {
+    id = meta.freeIds_.back();
+    meta.freeIds_.pop_back();
+  } else {
+    id = meta.nextId_++;
+  }
+
+  uint32_t old_id = ent->value.exchange(id);
+  DCHECK_EQ(old_id, kEntryIDInvalid);
+  return id;
+}
+
+void StaticMetaBase::destroy(EntryID* ent) {
+  try {
+    auto& meta = *this;
+    // Elements in other threads that use this id.
+    std::vector<ElementWrapper> elements;
+    {
+      std::lock_guard<std::mutex> g(meta.lock_);
+      uint32_t id = ent->value.exchange(kEntryIDInvalid);
+      if (id == kEntryIDInvalid) {
+        return;
+      }
+
+      for (ThreadEntry* e = meta.head_.next; e != &meta.head_; e = e->next) {
+        if (id < e->elementsCapacity && e->elements[id].ptr) {
+          elements.push_back(e->elements[id]);
+
+          /*
+           * Writing another thread's ThreadEntry from here is fine;
+           * the only other potential reader is the owning thread --
+           * from onThreadExit (which grabs the lock, so is properly
+           * synchronized with us) or from get(), which also grabs
+           * the lock if it needs to resize the elements vector.
+           *
+           * We can't conflict with reads for a get(id), because
+           * it's illegal to call get on a thread local that's
+           * destructing.
+           */
+          e->elements[id].ptr = nullptr;
+          e->elements[id].deleter1 = nullptr;
+          e->elements[id].ownsDeleter = false;
+        }
+      }
+      meta.freeIds_.push_back(id);
+    }
+    // Delete elements outside the lock
+    for (ElementWrapper& elem : elements) {
+      elem.dispose(TLPDestructionMode::ALL_THREADS);
+    }
+  } catch (...) { // Just in case we get a lock error or something anyway...
+    LOG(WARNING) << "Destructor discarding an exception that was thrown.";
+  }
+}
+
+/**
+ * Reserve enough space in the ThreadEntry::elements for the item
+ * @id to fit in.
+ */
+void StaticMetaBase::reserve(EntryID* id) {
+  auto& meta = *this;
+  ThreadEntry* threadEntry = (*threadEntry_)();
+  size_t prevCapacity = threadEntry->elementsCapacity;
+
+  uint32_t idval = id->getOrAllocate(meta);
+  if (prevCapacity > idval) {
+    return;
+  }
+  // Growth factor < 2, see folly/docs/FBVector.md; + 5 to prevent
+  // very slow start.
+  size_t newCapacity = static_cast<size_t>((idval + 5) * 1.7);
+  assert(newCapacity > prevCapacity);
+  ElementWrapper* reallocated = nullptr;
+
+  // Need to grow. Note that we can't call realloc, as elements is
+  // still linked in meta, so another thread might access invalid memory
+  // after realloc succeeds. We'll copy by hand and update our ThreadEntry
+  // under the lock.
+  if (usingJEMalloc()) {
+    bool success = false;
+    size_t newByteSize = nallocx(newCapacity * sizeof(ElementWrapper), 0);
+
+    // Try to grow in place.
+    //
+    // Note that xallocx(MALLOCX_ZERO) will only zero newly allocated memory,
+    // even if a previous allocation allocated more than we requested.
+    // This is fine; we always use MALLOCX_ZERO with jemalloc and we
+    // always expand our allocation to the real size.
+    if (prevCapacity * sizeof(ElementWrapper) >= jemallocMinInPlaceExpandable) {
+      success =
+          (xallocx(threadEntry->elements, newByteSize, 0, MALLOCX_ZERO) ==
+           newByteSize);
+    }
+
+    // In-place growth failed.
+    if (!success) {
+      success =
+          ((reallocated = static_cast<ElementWrapper*>(
+                mallocx(newByteSize, MALLOCX_ZERO))) != nullptr);
+    }
+
+    if (success) {
+      // Expand to real size
+      assert(newByteSize / sizeof(ElementWrapper) >= newCapacity);
+      newCapacity = newByteSize / sizeof(ElementWrapper);
+    } else {
+      throw std::bad_alloc();
+    }
+  } else { // no jemalloc
+    // calloc() is simpler than malloc() followed by memset(), and
+    // potentially faster when dealing with a lot of memory, as it can get
+    // already-zeroed pages from the kernel.
+    reallocated = static_cast<ElementWrapper*>(
+        calloc(newCapacity, sizeof(ElementWrapper)));
+    if (!reallocated) {
+      throw std::bad_alloc();
+    }
+  }
+
+  // Success, update the entry
+  {
+    std::lock_guard<std::mutex> g(meta.lock_);
+
+    if (prevCapacity == 0) {
+      meta.push_back(threadEntry);
+    }
+
+    if (reallocated) {
+      /*
+       * Note: we need to hold the meta lock when copying data out of
+       * the old vector, because some other thread might be
+       * destructing a ThreadLocal and writing to the elements vector
+       * of this thread.
+       */
+      if (prevCapacity != 0) {
+        memcpy(
+            reallocated,
+            threadEntry->elements,
+            sizeof(*reallocated) * prevCapacity);
+      }
+      std::swap(reallocated, threadEntry->elements);
+    }
+    threadEntry->elementsCapacity = newCapacity;
+  }
+
+  free(reallocated);
+}
+
+ElementWrapper& StaticMetaBase::get(EntryID* ent) {
+  ThreadEntry* threadEntry = (*threadEntry_)();
+  uint32_t id = ent->getOrInvalid();
+  // if id is invalid, it is equal to uint32_t's max value.
+  // x <= max value is always true
+  if (UNLIKELY(threadEntry->elementsCapacity <= id)) {
+    reserve(ent);
+    id = ent->getOrInvalid();
+    assert(threadEntry->elementsCapacity > id);
+  }
+  return threadEntry->elements[id];
+}
+
 MAX_STATIC_CONSTRUCTOR_PRIORITY
 PthreadKeyUnregister PthreadKeyUnregister::instance_;
-
 }}
index e9843269a2ee6937816c2316fe63127b920f9e44..7eeb1f581780a5eb48632004428540e9b2c8a423 100644 (file)
@@ -20,6 +20,8 @@
 #include <limits.h>
 #include <pthread.h>
 
+#include <atomic>
+#include <functional>
 #include <mutex>
 #include <string>
 #include <vector>
 namespace folly {
 namespace threadlocal_detail {
 
-/**
- * Base class for deleters.
- */
-class DeleterBase {
- public:
-  virtual ~DeleterBase() { }
-  virtual void dispose(void* ptr, TLPDestructionMode mode) const = 0;
-};
-
-/**
- * Simple deleter class that calls delete on the passed-in pointer.
- */
-template <class Ptr>
-class SimpleDeleter : public DeleterBase {
- public:
-  virtual void dispose(void* ptr, TLPDestructionMode /*mode*/) const {
-    delete static_cast<Ptr>(ptr);
-  }
-};
-
-/**
- * Custom deleter that calls a given callable.
- */
-template <class Ptr, class Deleter>
-class CustomDeleter : public DeleterBase {
- public:
-  explicit CustomDeleter(Deleter d) : deleter_(d) { }
-  virtual void dispose(void* ptr, TLPDestructionMode mode) const {
-    deleter_(static_cast<Ptr>(ptr), mode);
-  }
- private:
-  Deleter deleter_;
-};
-
-
 /**
  * POD wrapper around an element (a void*) and an associated deleter.
  * This must be POD, as we memset() it to 0 and memcpy() it around.
  */
 struct ElementWrapper {
+  using DeleterFunType = void(void*, TLPDestructionMode);
+
   bool dispose(TLPDestructionMode mode) {
     if (ptr == nullptr) {
       return false;
     }
 
-    DCHECK(deleter != nullptr);
-    deleter->dispose(ptr, mode);
+    DCHECK(deleter1 != nullptr);
+    ownsDeleter ? (*deleter2)(ptr, mode) : (*deleter1)(ptr, mode);
     cleanup();
     return true;
   }
@@ -112,16 +81,12 @@ struct ElementWrapper {
   template <class Ptr>
   void set(Ptr p) {
     DCHECK(ptr == nullptr);
-    DCHECK(deleter == nullptr);
+    DCHECK(deleter1 == nullptr);
 
     if (p) {
-      // We leak a single object here but that is ok.  If we used an
-      // object directly, there is a chance that the destructor will be
-      // called on that static object before any of the ElementWrappers
-      // are disposed and that isn't so nice.
-      static auto d = new SimpleDeleter<Ptr>();
       ptr = p;
-      deleter = d;
+      deleter1 =
+          +[](void* pt, TLPDestructionMode) { delete static_cast<Ptr>(pt); };
       ownsDeleter = false;
     }
   }
@@ -129,28 +94,36 @@ struct ElementWrapper {
   template <class Ptr, class Deleter>
   void set(Ptr p, Deleter d) {
     DCHECK(ptr == nullptr);
-    DCHECK(deleter == nullptr);
+    DCHECK(deleter2 == nullptr);
     if (p) {
       ptr = p;
-      deleter = new CustomDeleter<Ptr,Deleter>(d);
+      deleter2 = new std::function<DeleterFunType>(
+          [d](void* pt, TLPDestructionMode mode) {
+            d(static_cast<Ptr>(pt), mode);
+          });
       ownsDeleter = true;
     }
   }
 
   void cleanup() {
     if (ownsDeleter) {
-      delete deleter;
+      delete deleter2;
     }
     ptr = nullptr;
-    deleter = nullptr;
+    deleter1 = nullptr;
     ownsDeleter = false;
   }
 
   void* ptr;
-  DeleterBase* deleter;
+  union {
+    DeleterFunType* deleter1;
+    std::function<DeleterFunType>* deleter2;
+  };
   bool ownsDeleter;
 };
 
+struct StaticMetaBase;
+
 /**
  * Per-thread entry.  Each thread using a StaticMeta object has one.
  * This is written from the owning thread only (under the lock), read
@@ -162,6 +135,7 @@ struct ThreadEntry {
   size_t elementsCapacity{0};
   ThreadEntry* next{nullptr};
   ThreadEntry* prev{nullptr};
+  StaticMetaBase* meta{nullptr};
 };
 
 constexpr uint32_t kEntryIDInvalid = std::numeric_limits<uint32_t>::max();
@@ -220,15 +194,7 @@ class PthreadKeyUnregister {
   static PthreadKeyUnregister instance_;
 };
 
-// Held in a singleton to track our global instances.
-// We have one of these per "Tag", by default one for the whole system
-// (Tag=void).
-//
-// Creating and destroying ThreadLocalPtr objects, as well as thread exit
-// for threads that use ThreadLocalPtr objects collide on a lock inside
-// StaticMeta; you can specify multiple Tag types to break that lock.
-template <class Tag>
-struct StaticMeta {
+struct StaticMetaBase {
   // Represents an ID of a thread local object. Initially set to the maximum
   // uint. This representation allows us to avoid a branch in accessing TLS data
   // (because if you test capacity > id if id = maxint then the test will always
@@ -262,28 +228,21 @@ struct StaticMeta {
       return value.load(std::memory_order_relaxed);
     }
 
-    uint32_t getOrAllocate() {
+    uint32_t getOrAllocate(StaticMetaBase& meta) {
       uint32_t id = getOrInvalid();
       if (id != kEntryIDInvalid) {
         return id;
       }
       // The lock inside allocate ensures that a single value is allocated
-      return instance().allocate(this);
+      return meta.allocate(this);
     }
   };
 
-  static StaticMeta<Tag>& instance() {
-    // Leak it on exit, there's only one per process and we don't have to
-    // worry about synchronization with exiting threads.
-    static auto instance = detail::createGlobal<StaticMeta<Tag>, void>();
-    return *instance;
-  }
+  explicit StaticMetaBase(ThreadEntry* (*threadEntry)());
 
-  uint32_t nextId_;
-  std::vector<uint32_t> freeIds_;
-  std::mutex lock_;
-  pthread_key_t pthreadKey_;
-  ThreadEntry head_;
+  ~StaticMetaBase() {
+    LOG(FATAL) << "StaticMeta lives forever!";
+  }
 
   void push_back(ThreadEntry* t) {
     t->next = &head_;
@@ -298,16 +257,43 @@ struct StaticMeta {
     t->next = t->prev = t;
   }
 
-  StaticMeta() : nextId_(1) {
-    head_.next = head_.prev = &head_;
-    int ret = pthread_key_create(&pthreadKey_, &onThreadExit);
-    checkPosixError(ret, "pthread_key_create failed");
-    PthreadKeyUnregister::registerKey(pthreadKey_);
+  static void onThreadExit(void* ptr);
+
+  uint32_t allocate(EntryID* ent);
 
+  void destroy(EntryID* ent);
+
+  /**
+   * Reserve enough space in the ThreadEntry::elements for the item
+   * @id to fit in.
+   */
+  void reserve(EntryID* id);
+
+  ElementWrapper& get(EntryID* ent);
+
+  uint32_t nextId_;
+  std::vector<uint32_t> freeIds_;
+  std::mutex lock_;
+  pthread_key_t pthreadKey_;
+  ThreadEntry head_;
+  ThreadEntry* (*threadEntry_)();
+};
+
+// Held in a singleton to track our global instances.
+// We have one of these per "Tag", by default one for the whole system
+// (Tag=void).
+//
+// Creating and destroying ThreadLocalPtr objects, as well as thread exit
+// for threads that use ThreadLocalPtr objects collide on a lock inside
+// StaticMeta; you can specify multiple Tag types to break that lock.
+template <class Tag>
+struct StaticMeta : StaticMetaBase {
+  StaticMeta() : StaticMetaBase(&StaticMeta::getThreadEntry) {
 #if FOLLY_HAVE_PTHREAD_ATFORK
-    ret = pthread_atfork(/*prepare*/ &StaticMeta::preFork,
-                         /*parent*/ &StaticMeta::onForkParent,
-                         /*child*/ &StaticMeta::onForkChild);
+    int ret = pthread_atfork(
+        /*prepare*/ &StaticMeta::preFork,
+        /*parent*/ &StaticMeta::onForkParent,
+        /*child*/ &StaticMeta::onForkChild);
     checkPosixError(ret, "pthread_atfork failed");
 #elif !__ANDROID__ && !defined(_MSC_VER)
     // pthread_atfork is not part of the Android NDK at least as of n9d. If
@@ -318,18 +304,24 @@ struct StaticMeta {
     #warning pthread_atfork unavailable
 #endif
   }
-  ~StaticMeta() {
-    LOG(FATAL) << "StaticMeta lives forever!";
+
+  static StaticMeta<Tag>& instance() {
+    // Leak it on exit, there's only one per process and we don't have to
+    // worry about synchronization with exiting threads.
+    static auto instance = detail::createGlobal<StaticMeta<Tag>, void>();
+    return *instance;
   }
 
   static ThreadEntry* getThreadEntrySlow() {
-    auto key = instance().pthreadKey_;
+    auto& meta = instance();
+    auto key = meta.pthreadKey_;
     ThreadEntry* threadEntry =
       static_cast<ThreadEntry*>(pthread_getspecific(key));
     if (!threadEntry) {
-        threadEntry = new ThreadEntry();
-        int ret = pthread_setspecific(key, threadEntry);
-        checkPosixError(ret, "pthread_setspecific failed");
+      threadEntry = new ThreadEntry();
+      threadEntry->meta = &meta;
+      int ret = pthread_setspecific(key, threadEntry);
+      checkPosixError(ret, "pthread_setspecific failed");
     }
     return threadEntry;
   }
@@ -362,212 +354,6 @@ struct StaticMeta {
     }
     instance().lock_.unlock();
   }
-
-  static void onThreadExit(void* ptr) {
-    auto& meta = instance();
-
-    // pthread sets the thread-specific value corresponding
-    // to meta.pthreadKey_ to NULL before calling onThreadExit.
-    // We need to set it back to ptr to enable the correct behaviour
-    // of the subsequent calls of getThreadEntry
-    // (which may happen in user-provided custom deleters)
-    pthread_setspecific(meta.pthreadKey_, ptr);
-
-    ThreadEntry* threadEntry = getThreadEntry();
-    DCHECK_GT(threadEntry->elementsCapacity, 0);
-
-    {
-      std::lock_guard<std::mutex> g(meta.lock_);
-      meta.erase(threadEntry);
-      // No need to hold the lock any longer; the ThreadEntry is private to this
-      // thread now that it's been removed from meta.
-    }
-    // NOTE: User-provided deleter / object dtor itself may be using ThreadLocal
-    // with the same Tag, so dispose() calls below may (re)create some of the
-    // elements or even increase elementsCapacity, thus multiple cleanup rounds
-    // may be required.
-    for (bool shouldRun = true; shouldRun; ) {
-      shouldRun = false;
-      FOR_EACH_RANGE(i, 0, threadEntry->elementsCapacity) {
-        if (threadEntry->elements[i].dispose(TLPDestructionMode::THIS_THREAD)) {
-          shouldRun = true;
-        }
-      }
-    }
-    free(threadEntry->elements);
-    threadEntry->elements = nullptr;
-    pthread_setspecific(meta.pthreadKey_, nullptr);
-
-    delete threadEntry;
-  }
-
-  static uint32_t allocate(EntryID* ent) {
-    uint32_t id;
-    auto & meta = instance();
-    std::lock_guard<std::mutex> g(meta.lock_);
-
-    id = ent->value.load();
-    if (id != kEntryIDInvalid) {
-      return id;
-    }
-
-    if (!meta.freeIds_.empty()) {
-      id = meta.freeIds_.back();
-      meta.freeIds_.pop_back();
-    } else {
-      id = meta.nextId_++;
-    }
-
-    uint32_t old_id = ent->value.exchange(id);
-    DCHECK_EQ(old_id, kEntryIDInvalid);
-    return id;
-  }
-
-  static void destroy(EntryID* ent) {
-    try {
-      auto & meta = instance();
-      // Elements in other threads that use this id.
-      std::vector<ElementWrapper> elements;
-      {
-        std::lock_guard<std::mutex> g(meta.lock_);
-        uint32_t id = ent->value.exchange(kEntryIDInvalid);
-        if (id == kEntryIDInvalid) {
-          return;
-        }
-
-        for (ThreadEntry* e = meta.head_.next; e != &meta.head_; e = e->next) {
-          if (id < e->elementsCapacity && e->elements[id].ptr) {
-            elements.push_back(e->elements[id]);
-
-            /*
-             * Writing another thread's ThreadEntry from here is fine;
-             * the only other potential reader is the owning thread --
-             * from onThreadExit (which grabs the lock, so is properly
-             * synchronized with us) or from get(), which also grabs
-             * the lock if it needs to resize the elements vector.
-             *
-             * We can't conflict with reads for a get(id), because
-             * it's illegal to call get on a thread local that's
-             * destructing.
-             */
-            e->elements[id].ptr = nullptr;
-            e->elements[id].deleter = nullptr;
-            e->elements[id].ownsDeleter = false;
-          }
-        }
-        meta.freeIds_.push_back(id);
-      }
-      // Delete elements outside the lock
-      FOR_EACH(it, elements) {
-        it->dispose(TLPDestructionMode::ALL_THREADS);
-      }
-    } catch (...) { // Just in case we get a lock error or something anyway...
-      LOG(WARNING) << "Destructor discarding an exception that was thrown.";
-    }
-  }
-
-  /**
-   * Reserve enough space in the ThreadEntry::elements for the item
-   * @id to fit in.
-   */
-  static void reserve(EntryID* id) {
-    auto& meta = instance();
-    ThreadEntry* threadEntry = getThreadEntry();
-    size_t prevCapacity = threadEntry->elementsCapacity;
-
-    uint32_t idval = id->getOrAllocate();
-    if (prevCapacity > idval) {
-      return;
-    }
-    // Growth factor < 2, see folly/docs/FBVector.md; + 5 to prevent
-    // very slow start.
-    size_t newCapacity = static_cast<size_t>((idval + 5) * 1.7);
-    assert(newCapacity > prevCapacity);
-    ElementWrapper* reallocated = nullptr;
-
-    // Need to grow. Note that we can't call realloc, as elements is
-    // still linked in meta, so another thread might access invalid memory
-    // after realloc succeeds. We'll copy by hand and update our ThreadEntry
-    // under the lock.
-    if (usingJEMalloc()) {
-      bool success = false;
-      size_t newByteSize = nallocx(newCapacity * sizeof(ElementWrapper), 0);
-
-      // Try to grow in place.
-      //
-      // Note that xallocx(MALLOCX_ZERO) will only zero newly allocated memory,
-      // even if a previous allocation allocated more than we requested.
-      // This is fine; we always use MALLOCX_ZERO with jemalloc and we
-      // always expand our allocation to the real size.
-      if (prevCapacity * sizeof(ElementWrapper) >=
-          jemallocMinInPlaceExpandable) {
-        success = (xallocx(threadEntry->elements, newByteSize, 0, MALLOCX_ZERO)
-                   == newByteSize);
-      }
-
-      // In-place growth failed.
-      if (!success) {
-        success = ((reallocated = static_cast<ElementWrapper*>(
-                    mallocx(newByteSize, MALLOCX_ZERO))) != nullptr);
-      }
-
-      if (success) {
-        // Expand to real size
-        assert(newByteSize / sizeof(ElementWrapper) >= newCapacity);
-        newCapacity = newByteSize / sizeof(ElementWrapper);
-      } else {
-        throw std::bad_alloc();
-      }
-    } else {  // no jemalloc
-      // calloc() is simpler than malloc() followed by memset(), and
-      // potentially faster when dealing with a lot of memory, as it can get
-      // already-zeroed pages from the kernel.
-      reallocated = static_cast<ElementWrapper*>(
-          calloc(newCapacity, sizeof(ElementWrapper)));
-      if (!reallocated) {
-        throw std::bad_alloc();
-      }
-    }
-
-    // Success, update the entry
-    {
-      std::lock_guard<std::mutex> g(meta.lock_);
-
-      if (prevCapacity == 0) {
-        meta.push_back(threadEntry);
-      }
-
-      if (reallocated) {
-       /*
-        * Note: we need to hold the meta lock when copying data out of
-        * the old vector, because some other thread might be
-        * destructing a ThreadLocal and writing to the elements vector
-        * of this thread.
-        */
-        if (prevCapacity != 0) {
-          memcpy(reallocated, threadEntry->elements,
-                 sizeof(*reallocated) * prevCapacity);
-        }
-        std::swap(reallocated, threadEntry->elements);
-      }
-      threadEntry->elementsCapacity = newCapacity;
-    }
-
-    free(reallocated);
-  }
-
-  static ElementWrapper& get(EntryID* ent) {
-    ThreadEntry* threadEntry = getThreadEntry();
-    uint32_t id = ent->getOrInvalid();
-    // if id is invalid, it is equal to uint32_t's max value.
-    // x <= max value is always true
-    if (UNLIKELY(threadEntry->elementsCapacity <= id)) {
-      reserve(ent);
-      id = ent->getOrInvalid();
-      assert(threadEntry->elementsCapacity > id);
-    }
-    return threadEntry->elements[id];
-  }
 };
 
 }  // namespace threadlocal_detail