[compiler-rt] r361159 - [scudo][standalone] Introduce the Primary(s) and LocalCache

Kostya Kortchinsky via llvm-commits llvm-commits at lists.llvm.org
Mon May 20 07:40:04 PDT 2019


Author: cryptoad
Date: Mon May 20 07:40:04 2019
New Revision: 361159

URL: http://llvm.org/viewvc/llvm-project?rev=361159&view=rev
Log:
[scudo][standalone] Introduce the Primary(s) and LocalCache

Summary:
This CL introduces the 32 & 64-bit primary allocators, and associated
Local Cache. While the general idea is mostly similar to what exists
in sanitizer_common, it departs from the original code somewhat
significantly:
- the 64-bit primary no longer uses a free array at the end of a region
  but uses batches of free blocks in region 0, allowing for a
  convergence with the 32-bit primary behavior;
- as a result, there is only one (templated) local cache type for both
  primary allocators, and memory reclaiming can be implemented similarly
  for the 32-bit & 64-bit platforms;
- 64-bit primary regions are handled a bit differently: we do not
  reserve 4TB of memory that we split, but reserve `NumClasses *
  2^RegionSizeLog`, each region being offseted by a random number of
  pages from its computed base. A side effect of this is that the 64-bit
  primary works on 32-bit platform (I don't think we want to encourage
  it but it's an interesting side effect);

Reviewers: vitalybuka, eugenis, morehouse, hctim

Reviewed By: morehouse

Subscribers: srhines, mgorny, delcypher, jfb, #sanitizers, llvm-commits

Tags: #llvm, #sanitizers

Differential Revision: https://reviews.llvm.org/D61745

Added:
    compiler-rt/trunk/lib/scudo/standalone/local_cache.h
    compiler-rt/trunk/lib/scudo/standalone/primary32.h
    compiler-rt/trunk/lib/scudo/standalone/primary64.h
    compiler-rt/trunk/lib/scudo/standalone/tests/primary_test.cc
Modified:
    compiler-rt/trunk/lib/scudo/standalone/CMakeLists.txt
    compiler-rt/trunk/lib/scudo/standalone/tests/CMakeLists.txt
    compiler-rt/trunk/lib/scudo/standalone/tests/secondary_test.cc

Modified: compiler-rt/trunk/lib/scudo/standalone/CMakeLists.txt
URL: http://llvm.org/viewvc/llvm-project/compiler-rt/trunk/lib/scudo/standalone/CMakeLists.txt?rev=361159&r1=361158&r2=361159&view=diff
==============================================================================
--- compiler-rt/trunk/lib/scudo/standalone/CMakeLists.txt (original)
+++ compiler-rt/trunk/lib/scudo/standalone/CMakeLists.txt Mon May 20 07:40:04 2019
@@ -68,8 +68,11 @@ set(SCUDO_HEADERS
   internal_defs.h
   linux.h
   list.h
+  local_cache.h
   mutex.h
   platform.h
+  primary32.h
+  primary64.h
   quarantine.h
   release.h
   report.h

Added: compiler-rt/trunk/lib/scudo/standalone/local_cache.h
URL: http://llvm.org/viewvc/llvm-project/compiler-rt/trunk/lib/scudo/standalone/local_cache.h?rev=361159&view=auto
==============================================================================
--- compiler-rt/trunk/lib/scudo/standalone/local_cache.h (added)
+++ compiler-rt/trunk/lib/scudo/standalone/local_cache.h Mon May 20 07:40:04 2019
@@ -0,0 +1,178 @@
+//===-- local_cache.h -------------------------------------------*- C++ -*-===//
+//
+// Part of the LLVM Project, under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+//
+//===----------------------------------------------------------------------===//
+
+#ifndef SCUDO_LOCAL_CACHE_H_
+#define SCUDO_LOCAL_CACHE_H_
+
+#include "internal_defs.h"
+#include "stats.h"
+
+namespace scudo {
+
+template <class SizeClassAllocator> struct SizeClassAllocatorLocalCache {
+  typedef typename SizeClassAllocator::SizeClassMap SizeClassMap;
+
+  struct TransferBatch {
+    static const u32 MaxNumCached = SizeClassMap::MaxNumCachedHint;
+    void setFromArray(void **Array, u32 N) {
+      DCHECK_LE(N, MaxNumCached);
+      for (u32 I = 0; I < N; I++)
+        Batch[I] = Array[I];
+      Count = N;
+    }
+    void clear() { Count = 0; }
+    void add(void *P) {
+      DCHECK_LT(Count, MaxNumCached);
+      Batch[Count++] = P;
+    }
+    void copyToArray(void **Array) const {
+      for (u32 I = 0; I < Count; I++)
+        Array[I] = Batch[I];
+    }
+    u32 getCount() const { return Count; }
+    void *get(u32 I) const {
+      DCHECK_LE(I, Count);
+      return Batch[I];
+    }
+    static u32 MaxCached(uptr Size) {
+      return Min(MaxNumCached, SizeClassMap::getMaxCachedHint(Size));
+    }
+    TransferBatch *Next;
+
+  private:
+    u32 Count;
+    void *Batch[MaxNumCached];
+  };
+
+  void initLinkerInitialized(GlobalStats *S, SizeClassAllocator *A) {
+    Stats.initLinkerInitialized();
+    if (S)
+      S->link(&Stats);
+    Allocator = A;
+  }
+
+  void init(GlobalStats *S, SizeClassAllocator *A) {
+    memset(this, 0, sizeof(*this));
+    initLinkerInitialized(S, A);
+  }
+
+  void destroy(GlobalStats *S) {
+    drain();
+    if (S)
+      S->unlink(&Stats);
+  }
+
+  void *allocate(uptr ClassId) {
+    CHECK_LT(ClassId, NumClasses);
+    PerClass *C = &PerClassArray[ClassId];
+    if (C->Count == 0) {
+      if (UNLIKELY(!refill(C, ClassId)))
+        return nullptr;
+      DCHECK_GT(C->Count, 0);
+    }
+    // We read ClassSize first before accessing Chunks because it's adjacent to
+    // Count, while Chunks might be further off (depending on Count). That keeps
+    // the memory accesses in close quarters.
+    const uptr ClassSize = C->ClassSize;
+    void *P = C->Chunks[--C->Count];
+    // The jury is still out as to whether any kind of PREFETCH here increases
+    // performance. It definitely decreases performance on Android though.
+    // if (!SCUDO_ANDROID) PREFETCH(P);
+    Stats.add(StatAllocated, ClassSize);
+    return P;
+  }
+
+  void deallocate(uptr ClassId, void *P) {
+    CHECK_LT(ClassId, NumClasses);
+    PerClass *C = &PerClassArray[ClassId];
+    // We still have to initialize the cache in the event that the first heap
+    // operation in a thread is a deallocation.
+    initCacheMaybe(C);
+    if (C->Count == C->MaxCount)
+      drain(C, ClassId);
+    // See comment in allocate() about memory accesses.
+    const uptr ClassSize = C->ClassSize;
+    C->Chunks[C->Count++] = P;
+    Stats.sub(StatAllocated, ClassSize);
+  }
+
+  void drain() {
+    for (uptr I = 0; I < NumClasses; I++) {
+      PerClass *C = &PerClassArray[I];
+      while (C->Count > 0)
+        drain(C, I);
+    }
+  }
+
+  TransferBatch *createBatch(uptr ClassId, void *B) {
+    if (ClassId != SizeClassMap::BatchClassId)
+      B = allocate(SizeClassMap::BatchClassId);
+    return reinterpret_cast<TransferBatch *>(B);
+  }
+
+  LocalStats &getStats() { return Stats; }
+
+private:
+  static const uptr NumClasses = SizeClassMap::NumClasses;
+  struct PerClass {
+    u32 Count;
+    u32 MaxCount;
+    uptr ClassSize;
+    void *Chunks[2 * TransferBatch::MaxNumCached];
+  };
+  PerClass PerClassArray[NumClasses];
+  LocalStats Stats;
+  SizeClassAllocator *Allocator;
+
+  ALWAYS_INLINE void initCacheMaybe(PerClass *C) {
+    if (LIKELY(C->MaxCount))
+      return;
+    initCache();
+    DCHECK_NE(C->MaxCount, 0U);
+  }
+
+  NOINLINE void initCache() {
+    for (uptr I = 0; I < NumClasses; I++) {
+      PerClass *P = &PerClassArray[I];
+      const uptr Size = SizeClassAllocator::getSizeByClassId(I);
+      P->MaxCount = 2 * TransferBatch::MaxCached(Size);
+      P->ClassSize = Size;
+    }
+  }
+
+  void destroyBatch(uptr ClassId, void *B) {
+    if (ClassId != SizeClassMap::BatchClassId)
+      deallocate(SizeClassMap::BatchClassId, B);
+  }
+
+  NOINLINE bool refill(PerClass *C, uptr ClassId) {
+    initCacheMaybe(C);
+    TransferBatch *B = Allocator->popBatch(this, ClassId);
+    if (UNLIKELY(!B))
+      return false;
+    DCHECK_GT(B->getCount(), 0);
+    B->copyToArray(C->Chunks);
+    C->Count = B->getCount();
+    destroyBatch(ClassId, B);
+    return true;
+  }
+
+  NOINLINE void drain(PerClass *C, uptr ClassId) {
+    const u32 Count = Min(C->MaxCount / 2, C->Count);
+    const uptr FirstIndexToDrain = C->Count - Count;
+    TransferBatch *B = createBatch(ClassId, C->Chunks[FirstIndexToDrain]);
+    CHECK(B);
+    B->setFromArray(&C->Chunks[FirstIndexToDrain], Count);
+    C->Count -= Count;
+    Allocator->pushBatch(ClassId, B);
+  }
+};
+
+} // namespace scudo
+
+#endif // SCUDO_LOCAL_CACHE_H_

Added: compiler-rt/trunk/lib/scudo/standalone/primary32.h
URL: http://llvm.org/viewvc/llvm-project/compiler-rt/trunk/lib/scudo/standalone/primary32.h?rev=361159&view=auto
==============================================================================
--- compiler-rt/trunk/lib/scudo/standalone/primary32.h (added)
+++ compiler-rt/trunk/lib/scudo/standalone/primary32.h Mon May 20 07:40:04 2019
@@ -0,0 +1,388 @@
+//===-- primary32.h ---------------------------------------------*- C++ -*-===//
+//
+// Part of the LLVM Project, under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+//
+//===----------------------------------------------------------------------===//
+
+#ifndef SCUDO_PRIMARY32_H_
+#define SCUDO_PRIMARY32_H_
+
+#include "bytemap.h"
+#include "common.h"
+#include "list.h"
+#include "local_cache.h"
+#include "release.h"
+#include "report.h"
+#include "stats.h"
+#include "string_utils.h"
+
+namespace scudo {
+
+// SizeClassAllocator32 is an allocator for 32 or 64-bit address space.
+//
+// It maps Regions of 2^RegionSizeLog bytes aligned on a 2^RegionSizeLog bytes
+// boundary, and keeps a bytemap of the mappable address space to track the size
+// class they are associated with.
+//
+// Mapped regions are split into equally sized Blocks according to the size
+// class they belong to, and the associated pointers are shuffled to prevent any
+// predictable address pattern (the predictability increases with the block
+// size).
+//
+// Regions for size class 0 are special and used to hold TransferBatches, which
+// allow to transfer arrays of pointers from the global size class freelist to
+// the thread specific freelist for said class, and back.
+//
+// Memory used by this allocator is never unmapped but can be partially
+// reclaimed if the platform allows for it.
+
+template <class SizeClassMapT, uptr RegionSizeLog> class SizeClassAllocator32 {
+public:
+  typedef SizeClassMapT SizeClassMap;
+  // Regions should be large enough to hold the largest Block.
+  COMPILER_CHECK((1UL << RegionSizeLog) >= SizeClassMap::MaxSize);
+  typedef SizeClassAllocator32<SizeClassMapT, RegionSizeLog> ThisT;
+  typedef SizeClassAllocatorLocalCache<ThisT> CacheT;
+  typedef typename CacheT::TransferBatch TransferBatch;
+
+  static uptr getSizeByClassId(uptr ClassId) {
+    return (ClassId == SizeClassMap::BatchClassId)
+               ? sizeof(TransferBatch)
+               : SizeClassMap::getSizeByClassId(ClassId);
+  }
+
+  static bool canAllocate(uptr Size) { return Size <= SizeClassMap::MaxSize; }
+
+  void initLinkerInitialized(s32 ReleaseToOsInterval) {
+    if (SCUDO_FUCHSIA)
+      reportError("SizeClassAllocator32 is not supported on Fuchsia");
+
+    PossibleRegions.initLinkerInitialized();
+    MinRegionIndex = NumRegions; // MaxRegionIndex is already initialized to 0.
+
+    u32 Seed;
+    if (UNLIKELY(!getRandom(reinterpret_cast<void *>(&Seed), sizeof(Seed))))
+      Seed =
+          static_cast<u32>(getMonotonicTime() ^
+                           (reinterpret_cast<uptr>(SizeClassInfoArray) >> 6));
+    const uptr PageSize = getPageSizeCached();
+    for (uptr I = 0; I < NumClasses; I++) {
+      SizeClassInfo *Sci = getSizeClassInfo(I);
+      Sci->RandState = getRandomU32(&Seed);
+      // See comment in the 64-bit primary about releasing smaller size classes.
+      Sci->CanRelease = (ReleaseToOsInterval > 0) &&
+                        (I != SizeClassMap::BatchClassId) &&
+                        (getSizeByClassId(I) >= (PageSize / 32));
+    }
+    ReleaseToOsIntervalMs = ReleaseToOsInterval;
+  }
+  void init(s32 ReleaseToOsInterval) {
+    memset(this, 0, sizeof(*this));
+    initLinkerInitialized(ReleaseToOsInterval);
+  }
+
+  TransferBatch *popBatch(CacheT *C, uptr ClassId) {
+    DCHECK_LT(ClassId, NumClasses);
+    SizeClassInfo *Sci = getSizeClassInfo(ClassId);
+    BlockingMutexLock L(&Sci->Mutex);
+    TransferBatch *B = Sci->FreeList.front();
+    if (B)
+      Sci->FreeList.pop_front();
+    else {
+      B = populateFreeList(C, ClassId, Sci);
+      if (UNLIKELY(!B))
+        return nullptr;
+    }
+    DCHECK_GT(B->getCount(), 0);
+    Sci->Stats.PoppedBlocks += B->getCount();
+    return B;
+  }
+
+  void pushBatch(uptr ClassId, TransferBatch *B) {
+    DCHECK_LT(ClassId, NumClasses);
+    DCHECK_GT(B->getCount(), 0);
+    SizeClassInfo *Sci = getSizeClassInfo(ClassId);
+    BlockingMutexLock L(&Sci->Mutex);
+    Sci->FreeList.push_front(B);
+    Sci->Stats.PushedBlocks += B->getCount();
+    if (Sci->CanRelease)
+      releaseToOSMaybe(Sci, ClassId);
+  }
+
+  void disable() {
+    for (uptr I = 0; I < NumClasses; I++)
+      getSizeClassInfo(I)->Mutex.lock();
+  }
+
+  void enable() {
+    for (sptr I = static_cast<sptr>(NumClasses) - 1; I >= 0; I--)
+      getSizeClassInfo(I)->Mutex.unlock();
+  }
+
+  template <typename F> void iterateOverBlocks(F Callback) {
+    for (uptr I = MinRegionIndex; I <= MaxRegionIndex; I++)
+      if (PossibleRegions[I]) {
+        const uptr BlockSize = getSizeByClassId(PossibleRegions[I]);
+        const uptr From = I * RegionSize;
+        const uptr To = From + (RegionSize / BlockSize) * BlockSize;
+        for (uptr Block = From; Block < To; Block += BlockSize)
+          Callback(Block);
+      }
+  }
+
+  void printStats() {
+    // TODO(kostyak): get the RSS per region.
+    uptr TotalMapped = 0;
+    uptr PoppedBlocks = 0;
+    uptr PushedBlocks = 0;
+    for (uptr I = 0; I < NumClasses; I++) {
+      SizeClassInfo *Sci = getSizeClassInfo(I);
+      TotalMapped += Sci->AllocatedUser;
+      PoppedBlocks += Sci->Stats.PoppedBlocks;
+      PushedBlocks += Sci->Stats.PushedBlocks;
+    }
+    Printf("Stats: SizeClassAllocator32: %zuM mapped in %zu allocations; "
+           "remains %zu\n",
+           TotalMapped >> 20, PoppedBlocks, PoppedBlocks - PushedBlocks);
+    for (uptr I = 0; I < NumClasses; I++)
+      printStats(I, 0);
+  }
+
+  void releaseToOS() {
+    for (uptr I = 1; I < NumClasses; I++) {
+      SizeClassInfo *Sci = getSizeClassInfo(I);
+      BlockingMutexLock L(&Sci->Mutex);
+      releaseToOSMaybe(Sci, I, /*Force=*/true);
+    }
+  }
+
+private:
+  static const uptr NumClasses = SizeClassMap::NumClasses;
+  static const uptr RegionSize = 1UL << RegionSizeLog;
+  static const uptr NumRegions = SCUDO_MMAP_RANGE_SIZE >> RegionSizeLog;
+#if SCUDO_WORDSIZE == 32U
+  typedef FlatByteMap<NumRegions> ByteMap;
+#else
+  typedef TwoLevelByteMap<(NumRegions >> 12), 1UL << 12> ByteMap;
+#endif
+
+  struct SizeClassStats {
+    uptr PoppedBlocks;
+    uptr PushedBlocks;
+  };
+
+  struct ReleaseToOsInfo {
+    uptr PushedBlocksAtLastRelease;
+    uptr RangesReleased;
+    uptr LastReleasedBytes;
+    u64 LastReleaseAtNs;
+  };
+
+  struct ALIGNED(SCUDO_CACHE_LINE_SIZE) SizeClassInfo {
+    BlockingMutex Mutex;
+    IntrusiveList<TransferBatch> FreeList;
+    SizeClassStats Stats;
+    bool CanRelease;
+    u32 RandState;
+    uptr AllocatedUser;
+    ReleaseToOsInfo ReleaseInfo;
+  };
+  COMPILER_CHECK(sizeof(SizeClassInfo) % SCUDO_CACHE_LINE_SIZE == 0);
+
+  uptr computeRegionId(uptr Mem) {
+    const uptr Id = Mem >> RegionSizeLog;
+    CHECK_LT(Id, NumRegions);
+    return Id;
+  }
+
+  uptr allocateRegionSlow() {
+    uptr MapSize = 2 * RegionSize;
+    const uptr MapBase = reinterpret_cast<uptr>(
+        map(nullptr, MapSize, "scudo:primary", MAP_ALLOWNOMEM));
+    if (UNLIKELY(!MapBase))
+      return 0;
+    const uptr MapEnd = MapBase + MapSize;
+    uptr Region = MapBase;
+    if (isAligned(Region, RegionSize)) {
+      SpinMutexLock L(&RegionsStashMutex);
+      if (NumberOfStashedRegions < MaxStashedRegions)
+        RegionsStash[NumberOfStashedRegions++] = MapBase + RegionSize;
+      else
+        MapSize = RegionSize;
+    } else {
+      Region = roundUpTo(MapBase, RegionSize);
+      unmap(reinterpret_cast<void *>(MapBase), Region - MapBase);
+      MapSize = RegionSize;
+    }
+    const uptr End = Region + MapSize;
+    if (End != MapEnd)
+      unmap(reinterpret_cast<void *>(End), MapEnd - End);
+    return Region;
+  }
+
+  uptr allocateRegion(uptr ClassId) {
+    DCHECK_LT(ClassId, NumClasses);
+    uptr Region = 0;
+    {
+      SpinMutexLock L(&RegionsStashMutex);
+      if (NumberOfStashedRegions > 0)
+        Region = RegionsStash[--NumberOfStashedRegions];
+    }
+    if (!Region)
+      Region = allocateRegionSlow();
+    if (LIKELY(Region)) {
+      if (ClassId) {
+        const uptr RegionIndex = computeRegionId(Region);
+        if (RegionIndex < MinRegionIndex)
+          MinRegionIndex = RegionIndex;
+        if (RegionIndex > MaxRegionIndex)
+          MaxRegionIndex = RegionIndex;
+        PossibleRegions.set(RegionIndex, static_cast<u8>(ClassId));
+      }
+    }
+    return Region;
+  }
+
+  SizeClassInfo *getSizeClassInfo(uptr ClassId) {
+    DCHECK_LT(ClassId, NumClasses);
+    return &SizeClassInfoArray[ClassId];
+  }
+
+  bool populateBatches(CacheT *C, SizeClassInfo *Sci, uptr ClassId,
+                       TransferBatch **CurrentBatch, u32 MaxCount,
+                       void **PointersArray, u32 Count) {
+    if (ClassId != SizeClassMap::BatchClassId)
+      shuffle(PointersArray, Count, &Sci->RandState);
+    TransferBatch *B = *CurrentBatch;
+    for (uptr I = 0; I < Count; I++) {
+      if (B && B->getCount() == MaxCount) {
+        Sci->FreeList.push_back(B);
+        B = nullptr;
+      }
+      if (!B) {
+        B = C->createBatch(ClassId, PointersArray[I]);
+        if (UNLIKELY(!B))
+          return false;
+        B->clear();
+      }
+      B->add(PointersArray[I]);
+    }
+    *CurrentBatch = B;
+    return true;
+  }
+
+  NOINLINE TransferBatch *populateFreeList(CacheT *C, uptr ClassId,
+                                           SizeClassInfo *Sci) {
+    const uptr Region = allocateRegion(ClassId);
+    if (UNLIKELY(!Region))
+      return nullptr;
+    C->getStats().add(StatMapped, RegionSize);
+    const uptr Size = getSizeByClassId(ClassId);
+    const u32 MaxCount = TransferBatch::MaxCached(Size);
+    DCHECK_GT(MaxCount, 0);
+    const uptr NumberOfBlocks = RegionSize / Size;
+    DCHECK_GT(NumberOfBlocks, 0);
+    TransferBatch *B = nullptr;
+    constexpr uptr ShuffleArraySize = 48;
+    void *ShuffleArray[ShuffleArraySize];
+    u32 Count = 0;
+    const uptr AllocatedUser = NumberOfBlocks * Size;
+    for (uptr I = Region; I < Region + AllocatedUser; I += Size) {
+      ShuffleArray[Count++] = reinterpret_cast<void *>(I);
+      if (Count == ShuffleArraySize) {
+        if (UNLIKELY(!populateBatches(C, Sci, ClassId, &B, MaxCount,
+                                      ShuffleArray, Count)))
+          return nullptr;
+        Count = 0;
+      }
+    }
+    if (Count) {
+      if (UNLIKELY(!populateBatches(C, Sci, ClassId, &B, MaxCount, ShuffleArray,
+                                    Count)))
+        return nullptr;
+    }
+    DCHECK(B);
+    DCHECK_GT(B->getCount(), 0);
+    Sci->AllocatedUser += AllocatedUser;
+    if (Sci->CanRelease)
+      Sci->ReleaseInfo.LastReleaseAtNs = getMonotonicTime();
+    return B;
+  }
+
+  void printStats(uptr ClassId, uptr Rss) {
+    SizeClassInfo *Sci = getSizeClassInfo(ClassId);
+    if (Sci->AllocatedUser == 0)
+      return;
+    const uptr InUse = Sci->Stats.PoppedBlocks - Sci->Stats.PushedBlocks;
+    const uptr AvailableChunks = Sci->AllocatedUser / getSizeByClassId(ClassId);
+    Printf("  %02zu (%6zu): mapped: %6zuK popped: %7zu pushed: %7zu inuse: %6zu"
+           " avail: %6zu rss: %6zuK\n",
+           ClassId, getSizeByClassId(ClassId), Sci->AllocatedUser >> 10,
+           Sci->Stats.PoppedBlocks, Sci->Stats.PushedBlocks, InUse,
+           AvailableChunks, Rss >> 10);
+  }
+
+  NOINLINE void releaseToOSMaybe(SizeClassInfo *Sci, uptr ClassId,
+                                 bool Force = false) {
+    const uptr BlockSize = getSizeByClassId(ClassId);
+    const uptr PageSize = getPageSizeCached();
+
+    CHECK_GE(Sci->Stats.PoppedBlocks, Sci->Stats.PushedBlocks);
+    const uptr N = Sci->Stats.PoppedBlocks - Sci->Stats.PushedBlocks;
+    if (N * BlockSize < PageSize)
+      return; // No chance to release anything.
+    if ((Sci->Stats.PushedBlocks - Sci->ReleaseInfo.PushedBlocksAtLastRelease) *
+            BlockSize <
+        PageSize) {
+      return; // Nothing new to release.
+    }
+
+    if (!Force) {
+      const s32 IntervalMs = ReleaseToOsIntervalMs;
+      if (IntervalMs < 0)
+        return;
+      if (Sci->ReleaseInfo.LastReleaseAtNs + IntervalMs * 1000000ULL >
+          getMonotonicTime()) {
+        return; // Memory was returned recently.
+      }
+    }
+
+    // TODO(kostyak): currently not ideal as we loop over all regions and
+    // iterate multiple times over the same freelist if a ClassId spans multiple
+    // regions. But it will have to do for now.
+    for (uptr I = MinRegionIndex; I <= MaxRegionIndex; I++) {
+      if (PossibleRegions[I] == ClassId) {
+        ReleaseRecorder Recorder(I * RegionSize);
+        releaseFreeMemoryToOS(&Sci->FreeList, I * RegionSize,
+                              RegionSize / PageSize, BlockSize, &Recorder);
+        if (Recorder.getReleasedRangesCount() > 0) {
+          Sci->ReleaseInfo.PushedBlocksAtLastRelease = Sci->Stats.PushedBlocks;
+          Sci->ReleaseInfo.RangesReleased += Recorder.getReleasedRangesCount();
+          Sci->ReleaseInfo.LastReleasedBytes = Recorder.getReleasedBytes();
+        }
+      }
+    }
+    Sci->ReleaseInfo.LastReleaseAtNs = getMonotonicTime();
+  }
+
+  SizeClassInfo SizeClassInfoArray[NumClasses];
+
+  ByteMap PossibleRegions;
+  // Keep track of the lowest & highest regions allocated to avoid looping
+  // through the whole NumRegions.
+  uptr MinRegionIndex;
+  uptr MaxRegionIndex;
+  s32 ReleaseToOsIntervalMs;
+  // Unless several threads request regions simultaneously from different size
+  // classes, the stash rarely contains more than 1 entry.
+  static constexpr uptr MaxStashedRegions = 4;
+  StaticSpinMutex RegionsStashMutex;
+  uptr NumberOfStashedRegions;
+  uptr RegionsStash[MaxStashedRegions];
+};
+
+} // namespace scudo
+
+#endif // SCUDO_PRIMARY32_H_

Added: compiler-rt/trunk/lib/scudo/standalone/primary64.h
URL: http://llvm.org/viewvc/llvm-project/compiler-rt/trunk/lib/scudo/standalone/primary64.h?rev=361159&view=auto
==============================================================================
--- compiler-rt/trunk/lib/scudo/standalone/primary64.h (added)
+++ compiler-rt/trunk/lib/scudo/standalone/primary64.h Mon May 20 07:40:04 2019
@@ -0,0 +1,373 @@
+//===-- primary64.h ---------------------------------------------*- C++ -*-===//
+//
+// Part of the LLVM Project, under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+//
+//===----------------------------------------------------------------------===//
+
+#ifndef SCUDO_PRIMARY64_H_
+#define SCUDO_PRIMARY64_H_
+
+#include "bytemap.h"
+#include "common.h"
+#include "list.h"
+#include "local_cache.h"
+#include "release.h"
+#include "stats.h"
+#include "string_utils.h"
+
+namespace scudo {
+
+// SizeClassAllocator64 is an allocator tuned for 64-bit address space.
+//
+// It starts by reserving NumClasses * 2^RegionSizeLog bytes, equally divided in
+// Regions, specific to each size class. Note that the base of that mapping is
+// random (based to the platform specific map() capabilities), and that each
+// Region actually starts at a random offset from its base.
+//
+// Regions are mapped incrementally on demand to fulfill allocation requests,
+// those mappings being split into equally sized Blocks based on the size class
+// they belong to. The Blocks created are shuffled to prevent predictable
+// address patterns (the predictability increases with the size of the Blocks).
+//
+// The 1st Region (for size class 0) holds the TransferBatches. This is a
+// structure used to transfer arrays of available pointers from the class size
+// freelist to the thread specific freelist, and back.
+//
+// The memory used by this allocator is never unmapped, but can be partially
+// released it the platform allows for it.
+
+template <class SizeClassMapT, uptr RegionSizeLog> class SizeClassAllocator64 {
+public:
+  typedef SizeClassMapT SizeClassMap;
+  typedef SizeClassAllocator64<SizeClassMap, RegionSizeLog> ThisT;
+  typedef SizeClassAllocatorLocalCache<ThisT> CacheT;
+  typedef typename CacheT::TransferBatch TransferBatch;
+
+  static uptr getSizeByClassId(uptr ClassId) {
+    return (ClassId == SizeClassMap::BatchClassId)
+               ? sizeof(TransferBatch)
+               : SizeClassMap::getSizeByClassId(ClassId);
+  }
+
+  static bool canAllocate(uptr Size) { return Size <= SizeClassMap::MaxSize; }
+
+  void initLinkerInitialized(s32 ReleaseToOsInterval) {
+    // Reserve the space required for the Primary.
+    PrimaryBase = reinterpret_cast<uptr>(
+        map(nullptr, PrimarySize, "scudo:primary", MAP_NOACCESS, &Data));
+
+    RegionInfoArray = reinterpret_cast<RegionInfo *>(
+        map(nullptr, sizeof(RegionInfo) * NumClasses, "scudo:regioninfo"));
+    DCHECK_EQ(reinterpret_cast<uptr>(RegionInfoArray) % SCUDO_CACHE_LINE_SIZE,
+              0);
+
+    u32 Seed;
+    if (UNLIKELY(!getRandom(reinterpret_cast<void *>(&Seed), sizeof(Seed))))
+      Seed = static_cast<u32>(getMonotonicTime() ^ (PrimaryBase >> 12));
+    const uptr PageSize = getPageSizeCached();
+    for (uptr I = 0; I < NumClasses; I++) {
+      RegionInfo *Region = getRegionInfo(I);
+      // The actual start of a region is offseted by a random number of pages.
+      Region->RegionBeg =
+          getRegionBaseByClassId(I) + (getRandomModN(&Seed, 16) + 1) * PageSize;
+      // Releasing smaller size classes doesn't necessarily yield to a
+      // meaningful RSS impact: there are more blocks per page, they are
+      // randomized around, and thus pages are less likely to be entirely empty.
+      // On top of this, attempting to release those require more iterations and
+      // memory accesses which ends up being fairly costly. The current lower
+      // limit is mostly arbitrary and based on empirical observations.
+      // TODO(kostyak): make the lower limit a runtime option
+      Region->CanRelease = (ReleaseToOsInterval > 0) &&
+                           (I != SizeClassMap::BatchClassId) &&
+                           (getSizeByClassId(I) >= (PageSize / 32));
+      Region->RandState = getRandomU32(&Seed);
+    }
+    ReleaseToOsIntervalMs = ReleaseToOsInterval;
+  }
+  void init(s32 ReleaseToOsInterval) {
+    memset(this, 0, sizeof(*this));
+    initLinkerInitialized(ReleaseToOsInterval);
+  }
+
+  TransferBatch *popBatch(CacheT *C, uptr ClassId) {
+    DCHECK_LT(ClassId, NumClasses);
+    RegionInfo *Region = getRegionInfo(ClassId);
+    BlockingMutexLock L(&Region->Mutex);
+    TransferBatch *B = Region->FreeList.front();
+    if (B)
+      Region->FreeList.pop_front();
+    else {
+      B = populateFreeList(C, ClassId, Region);
+      if (UNLIKELY(!B))
+        return nullptr;
+    }
+    DCHECK_GT(B->getCount(), 0);
+    Region->Stats.PoppedBlocks += B->getCount();
+    return B;
+  }
+
+  void pushBatch(uptr ClassId, TransferBatch *B) {
+    DCHECK_GT(B->getCount(), 0);
+    RegionInfo *Region = getRegionInfo(ClassId);
+    BlockingMutexLock L(&Region->Mutex);
+    Region->FreeList.push_front(B);
+    Region->Stats.PushedBlocks += B->getCount();
+    if (Region->CanRelease)
+      releaseToOSMaybe(Region, ClassId);
+  }
+
+  void disable() {
+    for (uptr I = 0; I < NumClasses; I++)
+      getRegionInfo(I)->Mutex.lock();
+  }
+
+  void enable() {
+    for (sptr I = static_cast<sptr>(NumClasses) - 1; I >= 0; I--)
+      getRegionInfo(I)->Mutex.unlock();
+  }
+
+  template <typename F> void iterateOverBlocks(F Callback) const {
+    for (uptr I = 1; I < NumClasses; I++) {
+      const RegionInfo *Region = getRegionInfo(I);
+      const uptr BlockSize = getSizeByClassId(I);
+      const uptr From = Region->RegionBeg;
+      const uptr To = From + Region->AllocatedUser;
+      for (uptr Block = From; Block < To; Block += BlockSize)
+        Callback(Block);
+    }
+  }
+
+  void printStats() const {
+    // TODO(kostyak): get the RSS per region.
+    uptr TotalMapped = 0;
+    uptr PoppedBlocks = 0;
+    uptr PushedBlocks = 0;
+    for (uptr I = 0; I < NumClasses; I++) {
+      RegionInfo *Region = getRegionInfo(I);
+      if (Region->MappedUser)
+        TotalMapped += Region->MappedUser;
+      PoppedBlocks += Region->Stats.PoppedBlocks;
+      PushedBlocks += Region->Stats.PushedBlocks;
+    }
+    Printf("Stats: Primary64: %zuM mapped (%zuM rss) in %zu allocations; "
+           "remains %zu\n",
+           TotalMapped >> 20, 0, PoppedBlocks, PoppedBlocks - PushedBlocks);
+
+    for (uptr I = 0; I < NumClasses; I++)
+      printStats(I, 0);
+  }
+
+  void releaseToOS() {
+    for (uptr I = 1; I < NumClasses; I++) {
+      RegionInfo *Region = getRegionInfo(I);
+      BlockingMutexLock L(&Region->Mutex);
+      releaseToOSMaybe(Region, I, /*Force=*/true);
+    }
+  }
+
+private:
+  static const uptr RegionSize = 1UL << RegionSizeLog;
+  static const uptr NumClasses = SizeClassMap::NumClasses;
+  static const uptr PrimarySize = RegionSize * NumClasses;
+
+  // Call map for user memory with at least this size.
+  static const uptr MapSizeIncrement = 1UL << 16;
+
+  struct RegionStats {
+    uptr PoppedBlocks;
+    uptr PushedBlocks;
+  };
+
+  struct ReleaseToOsInfo {
+    uptr PushedBlocksAtLastRelease;
+    uptr RangesReleased;
+    uptr LastReleasedBytes;
+    u64 LastReleaseAtNs;
+  };
+
+  struct ALIGNED(SCUDO_CACHE_LINE_SIZE) RegionInfo {
+    BlockingMutex Mutex;
+    IntrusiveList<TransferBatch> FreeList;
+    RegionStats Stats;
+    bool CanRelease;
+    bool Exhausted;
+    u32 RandState;
+    uptr RegionBeg;
+    uptr MappedUser;    // Bytes mapped for user memory.
+    uptr AllocatedUser; // Bytes allocated for user memory.
+    MapPlatformData Data;
+    ReleaseToOsInfo ReleaseInfo;
+  };
+  COMPILER_CHECK(sizeof(RegionInfo) % SCUDO_CACHE_LINE_SIZE == 0);
+
+  uptr PrimaryBase;
+  RegionInfo *RegionInfoArray;
+  MapPlatformData Data;
+  s32 ReleaseToOsIntervalMs;
+
+  RegionInfo *getRegionInfo(uptr ClassId) const {
+    DCHECK_LT(ClassId, NumClasses);
+    return &RegionInfoArray[ClassId];
+  }
+
+  uptr getRegionBaseByClassId(uptr ClassId) const {
+    return PrimaryBase + (ClassId << RegionSizeLog);
+  }
+
+  bool populateBatches(CacheT *C, RegionInfo *Region, uptr ClassId,
+                       TransferBatch **CurrentBatch, u32 MaxCount,
+                       void **PointersArray, u32 Count) {
+    // No need to shuffle the batches size class.
+    if (ClassId != SizeClassMap::BatchClassId)
+      shuffle(PointersArray, Count, &Region->RandState);
+    TransferBatch *B = *CurrentBatch;
+    for (uptr I = 0; I < Count; I++) {
+      if (B && B->getCount() == MaxCount) {
+        Region->FreeList.push_back(B);
+        B = nullptr;
+      }
+      if (!B) {
+        B = C->createBatch(ClassId, PointersArray[I]);
+        if (UNLIKELY(!B))
+          return false;
+        B->clear();
+      }
+      B->add(PointersArray[I]);
+    }
+    *CurrentBatch = B;
+    return true;
+  }
+
+  NOINLINE TransferBatch *populateFreeList(CacheT *C, uptr ClassId,
+                                           RegionInfo *Region) {
+    const uptr Size = getSizeByClassId(ClassId);
+    const u32 MaxCount = TransferBatch::MaxCached(Size);
+
+    const uptr RegionBeg = Region->RegionBeg;
+    const uptr MappedUser = Region->MappedUser;
+    const uptr TotalUserBytes = Region->AllocatedUser + MaxCount * Size;
+    // Map more space for blocks, if necessary.
+    if (LIKELY(TotalUserBytes > MappedUser)) {
+      // Do the mmap for the user memory.
+      const uptr UserMapSize =
+          roundUpTo(TotalUserBytes - MappedUser, MapSizeIncrement);
+      const uptr RegionBase = RegionBeg - getRegionBaseByClassId(ClassId);
+      if (UNLIKELY(RegionBase + MappedUser + UserMapSize > RegionSize)) {
+        if (!Region->Exhausted) {
+          Region->Exhausted = true;
+          printStats();
+          Printf(
+              "Scudo OOM: The process has Exhausted %zuM for size class %zu.\n",
+              RegionSize >> 20, Size);
+        }
+        return nullptr;
+      }
+      if (MappedUser == 0)
+        Region->Data = Data;
+      if (UNLIKELY(!map(reinterpret_cast<void *>(RegionBeg + MappedUser),
+                        UserMapSize, "scudo:primary",
+                        MAP_ALLOWNOMEM | MAP_RESIZABLE, &Region->Data)))
+        return nullptr;
+      Region->MappedUser += UserMapSize;
+      C->getStats().add(StatMapped, UserMapSize);
+    }
+
+    const uptr NumberOfBlocks = Min(
+        8UL * MaxCount, (Region->MappedUser - Region->AllocatedUser) / Size);
+    DCHECK_GT(NumberOfBlocks, 0);
+
+    TransferBatch *B = nullptr;
+    constexpr uptr ShuffleArraySize = 48;
+    void *ShuffleArray[ShuffleArraySize];
+    u32 Count = 0;
+    const uptr P = RegionBeg + Region->AllocatedUser;
+    const uptr AllocatedUser = NumberOfBlocks * Size;
+    for (uptr I = P; I < P + AllocatedUser; I += Size) {
+      ShuffleArray[Count++] = reinterpret_cast<void *>(I);
+      if (Count == ShuffleArraySize) {
+        if (UNLIKELY(!populateBatches(C, Region, ClassId, &B, MaxCount,
+                                      ShuffleArray, Count)))
+          return nullptr;
+        Count = 0;
+      }
+    }
+    if (Count) {
+      if (UNLIKELY(!populateBatches(C, Region, ClassId, &B, MaxCount,
+                                    ShuffleArray, Count)))
+        return nullptr;
+    }
+    DCHECK(B);
+    CHECK_GT(B->getCount(), 0);
+
+    Region->AllocatedUser += AllocatedUser;
+    Region->Exhausted = false;
+    if (Region->CanRelease)
+      Region->ReleaseInfo.LastReleaseAtNs = getMonotonicTime();
+
+    return B;
+  }
+
+  void printStats(uptr ClassId, uptr Rss) const {
+    RegionInfo *Region = getRegionInfo(ClassId);
+    if (Region->MappedUser == 0)
+      return;
+    const uptr InUse = Region->Stats.PoppedBlocks - Region->Stats.PushedBlocks;
+    const uptr AvailableChunks =
+        Region->AllocatedUser / getSizeByClassId(ClassId);
+    Printf("%s %02zu (%6zu): mapped: %6zuK popped: %7zu pushed: %7zu inuse: "
+           "%6zu avail: %6zu rss: %6zuK releases: %6zu last released: %6zuK "
+           "region: 0x%zx (0x%zx)\n",
+           Region->Exhausted ? "F" : " ", ClassId, getSizeByClassId(ClassId),
+           Region->MappedUser >> 10, Region->Stats.PoppedBlocks,
+           Region->Stats.PushedBlocks, InUse, AvailableChunks, Rss >> 10,
+           Region->ReleaseInfo.RangesReleased,
+           Region->ReleaseInfo.LastReleasedBytes >> 10, Region->RegionBeg,
+           getRegionBaseByClassId(ClassId));
+  }
+
+  NOINLINE void releaseToOSMaybe(RegionInfo *Region, uptr ClassId,
+                                 bool Force = false) {
+    const uptr BlockSize = getSizeByClassId(ClassId);
+    const uptr PageSize = getPageSizeCached();
+
+    CHECK_GE(Region->Stats.PoppedBlocks, Region->Stats.PushedBlocks);
+    const uptr N = Region->Stats.PoppedBlocks - Region->Stats.PushedBlocks;
+    if (N * BlockSize < PageSize)
+      return; // No chance to release anything.
+    if ((Region->Stats.PushedBlocks -
+         Region->ReleaseInfo.PushedBlocksAtLastRelease) *
+            BlockSize <
+        PageSize) {
+      return; // Nothing new to release.
+    }
+
+    if (!Force) {
+      const s32 IntervalMs = ReleaseToOsIntervalMs;
+      if (IntervalMs < 0)
+        return;
+      if (Region->ReleaseInfo.LastReleaseAtNs + IntervalMs * 1000000ULL >
+          getMonotonicTime()) {
+        return; // Memory was returned recently.
+      }
+    }
+
+    ReleaseRecorder Recorder(Region->RegionBeg, &Region->Data);
+    releaseFreeMemoryToOS(&Region->FreeList, Region->RegionBeg,
+                          roundUpTo(Region->AllocatedUser, PageSize) / PageSize,
+                          BlockSize, &Recorder);
+
+    if (Recorder.getReleasedRangesCount() > 0) {
+      Region->ReleaseInfo.PushedBlocksAtLastRelease =
+          Region->Stats.PushedBlocks;
+      Region->ReleaseInfo.RangesReleased += Recorder.getReleasedRangesCount();
+      Region->ReleaseInfo.LastReleasedBytes = Recorder.getReleasedBytes();
+    }
+    Region->ReleaseInfo.LastReleaseAtNs = getMonotonicTime();
+  }
+};
+
+} // namespace scudo
+
+#endif // SCUDO_PRIMARY64_H_

Modified: compiler-rt/trunk/lib/scudo/standalone/tests/CMakeLists.txt
URL: http://llvm.org/viewvc/llvm-project/compiler-rt/trunk/lib/scudo/standalone/tests/CMakeLists.txt?rev=361159&r1=361158&r2=361159&view=diff
==============================================================================
--- compiler-rt/trunk/lib/scudo/standalone/tests/CMakeLists.txt (original)
+++ compiler-rt/trunk/lib/scudo/standalone/tests/CMakeLists.txt Mon May 20 07:40:04 2019
@@ -57,6 +57,7 @@ set(SCUDO_UNIT_TEST_SOURCES
   list_test.cc
   map_test.cc
   mutex_test.cc
+  primary_test.cc
   quarantine_test.cc
   release_test.cc
   report_test.cc

Added: compiler-rt/trunk/lib/scudo/standalone/tests/primary_test.cc
URL: http://llvm.org/viewvc/llvm-project/compiler-rt/trunk/lib/scudo/standalone/tests/primary_test.cc?rev=361159&view=auto
==============================================================================
--- compiler-rt/trunk/lib/scudo/standalone/tests/primary_test.cc (added)
+++ compiler-rt/trunk/lib/scudo/standalone/tests/primary_test.cc Mon May 20 07:40:04 2019
@@ -0,0 +1,176 @@
+//===-- primary_test.cc -----------------------------------------*- C++ -*-===//
+//
+// Part of the LLVM Project, under the Apache License v2.0 with LLVM Exceptions.
+// See https://llvm.org/LICENSE.txt for license information.
+// SPDX-License-Identifier: Apache-2.0 WITH LLVM-exception
+//
+//===----------------------------------------------------------------------===//
+
+#include "primary32.h"
+#include "primary64.h"
+#include "size_class_map.h"
+
+#include "gtest/gtest.h"
+
+#include <condition_variable>
+#include <mutex>
+#include <thread>
+
+// Note that with small enough regions, the SizeClassAllocator64 also works on
+// 32-bit architectures. It's not something we want to encourage, but we still
+// should ensure the tests pass.
+
+template <typename Primary> static void testPrimary() {
+  const scudo::uptr NumberOfAllocations = 32U;
+  std::unique_ptr<Primary> Allocator(new Primary);
+  Allocator->init(/*ReleaseToOsInterval=*/-1);
+  typename Primary::CacheT Cache;
+  Cache.init(nullptr, Allocator.get());
+  for (scudo::uptr I = 0; I <= 16U; I++) {
+    const scudo::uptr Size = 1UL << I;
+    if (!Primary::canAllocate(Size))
+      continue;
+    const scudo::uptr ClassId = Primary::SizeClassMap::getClassIdBySize(Size);
+    void *Pointers[NumberOfAllocations];
+    for (scudo::uptr J = 0; J < NumberOfAllocations; J++) {
+      void *P = Cache.allocate(ClassId);
+      memset(P, 'B', Size);
+      Pointers[J] = P;
+    }
+    for (scudo::uptr J = 0; J < NumberOfAllocations; J++)
+      Cache.deallocate(ClassId, Pointers[J]);
+  }
+  Cache.destroy(nullptr);
+  Allocator->releaseToOS();
+  Allocator->printStats();
+}
+
+TEST(ScudoPrimaryTest, BasicPrimary) {
+  using SizeClassMap = scudo::DefaultSizeClassMap;
+  testPrimary<scudo::SizeClassAllocator32<SizeClassMap, 24U>>();
+  testPrimary<scudo::SizeClassAllocator64<SizeClassMap, 24U>>();
+}
+
+// The 64-bit SizeClassAllocator can be easily OOM'd with small region sizes.
+// For the 32-bit one, it requires actually exhausting memory, so we skip it.
+TEST(ScudoPrimaryTest, Primary64OOM) {
+  using Primary = scudo::SizeClassAllocator64<scudo::DefaultSizeClassMap, 20U>;
+  using TransferBatch = Primary::CacheT::TransferBatch;
+  Primary Allocator;
+  Allocator.init(/*ReleaseToOsInterval=*/-1);
+  typename Primary::CacheT Cache;
+  scudo::GlobalStats Stats;
+  Stats.init();
+  Cache.init(&Stats, &Allocator);
+  bool AllocationFailed = false;
+  std::vector<TransferBatch *> Batches;
+  const scudo::uptr ClassId = Primary::SizeClassMap::LargestClassId;
+  const scudo::uptr Size = Primary::getSizeByClassId(ClassId);
+  for (scudo::uptr I = 0; I < 10000U; I++) {
+    TransferBatch *B = Allocator.popBatch(&Cache, ClassId);
+    if (!B) {
+      AllocationFailed = true;
+      break;
+    }
+    for (scudo::uptr J = 0; J < B->getCount(); J++)
+      memset(B->get(J), 'B', Size);
+    Batches.push_back(B);
+  }
+  while (!Batches.empty()) {
+    Allocator.pushBatch(ClassId, Batches.back());
+    Batches.pop_back();
+  }
+  Cache.destroy(nullptr);
+  Allocator.releaseToOS();
+  Allocator.printStats();
+  EXPECT_EQ(AllocationFailed, true);
+}
+
+template <typename Primary> static void testIteratePrimary() {
+  std::unique_ptr<Primary> Allocator(new Primary);
+  Allocator->init(/*ReleaseToOsInterval=*/-1);
+  typename Primary::CacheT Cache;
+  Cache.init(nullptr, Allocator.get());
+  std::vector<std::pair<scudo::uptr, void *>> V;
+  for (scudo::uptr I = 0; I < 64U; I++) {
+    const scudo::uptr Size = std::rand() % Primary::SizeClassMap::MaxSize;
+    const scudo::uptr ClassId = Primary::SizeClassMap::getClassIdBySize(Size);
+    void *P = Cache.allocate(ClassId);
+    V.push_back(std::make_pair(ClassId, P));
+  }
+  scudo::uptr Found = 0;
+  auto Lambda = [V, &Found](scudo::uptr Block) {
+    for (const auto &Pair : V) {
+      if (Pair.second == reinterpret_cast<void *>(Block))
+        Found++;
+    }
+  };
+  Allocator->disable();
+  Allocator->iterateOverBlocks(Lambda);
+  Allocator->enable();
+  EXPECT_EQ(Found, V.size());
+  while (!V.empty()) {
+    auto Pair = V.back();
+    Cache.deallocate(Pair.first, Pair.second);
+    V.pop_back();
+  }
+  Cache.destroy(nullptr);
+  Allocator->releaseToOS();
+  Allocator->printStats();
+}
+
+TEST(ScudoPrimaryTest, PrimaryIterate) {
+  using SizeClassMap = scudo::DefaultSizeClassMap;
+  testIteratePrimary<scudo::SizeClassAllocator32<SizeClassMap, 24U>>();
+  testIteratePrimary<scudo::SizeClassAllocator64<SizeClassMap, 24U>>();
+}
+
+static std::mutex Mutex;
+static std::condition_variable Cv;
+static bool Ready = false;
+
+template <typename Primary> static void performAllocations(Primary *Allocator) {
+  static THREADLOCAL typename Primary::CacheT Cache;
+  Cache.init(nullptr, Allocator);
+  std::vector<std::pair<scudo::uptr, void *>> V;
+  {
+    std::unique_lock<std::mutex> Lock(Mutex);
+    while (!Ready)
+      Cv.wait(Lock);
+  }
+  for (scudo::uptr I = 0; I < 256U; I++) {
+    const scudo::uptr Size = std::rand() % Primary::SizeClassMap::MaxSize;
+    const scudo::uptr ClassId = Primary::SizeClassMap::getClassIdBySize(Size);
+    void *P = Cache.allocate(ClassId);
+    V.push_back(std::make_pair(ClassId, P));
+  }
+  while (!V.empty()) {
+    auto Pair = V.back();
+    Cache.deallocate(Pair.first, Pair.second);
+    V.pop_back();
+  }
+  Cache.destroy(nullptr);
+}
+
+template <typename Primary> static void testPrimaryThreaded() {
+  std::unique_ptr<Primary> Allocator(new Primary);
+  Allocator->init(/*ReleaseToOsInterval=*/-1);
+  std::thread Threads[10];
+  for (scudo::uptr I = 0; I < 10U; I++)
+    Threads[I] = std::thread(performAllocations<Primary>, Allocator.get());
+  {
+    std::unique_lock<std::mutex> Lock(Mutex);
+    Ready = true;
+    Cv.notify_all();
+  }
+  for (auto &T : Threads)
+    T.join();
+  Allocator->releaseToOS();
+  Allocator->printStats();
+}
+
+TEST(ScudoPrimaryTest, PrimaryThreaded) {
+  using SizeClassMap = scudo::SvelteSizeClassMap;
+  testPrimaryThreaded<scudo::SizeClassAllocator32<SizeClassMap, 24U>>();
+  testPrimaryThreaded<scudo::SizeClassAllocator64<SizeClassMap, 24U>>();
+}

Modified: compiler-rt/trunk/lib/scudo/standalone/tests/secondary_test.cc
URL: http://llvm.org/viewvc/llvm-project/compiler-rt/trunk/lib/scudo/standalone/tests/secondary_test.cc?rev=361159&r1=361158&r2=361159&view=diff
==============================================================================
--- compiler-rt/trunk/lib/scudo/standalone/tests/secondary_test.cc (original)
+++ compiler-rt/trunk/lib/scudo/standalone/tests/secondary_test.cc Mon May 20 07:40:04 2019
@@ -38,7 +38,7 @@ TEST(ScudoSecondaryTest, SecondaryBasic)
   L->deallocate(P);
 
   std::vector<void *> V;
-  for (scudo::u8 I = 0; I < 32; I++)
+  for (scudo::uptr I = 0; I < 32U; I++)
     V.push_back(L->allocate(Size));
   std::random_shuffle(V.begin(), V.end());
   while (!V.empty()) {
@@ -84,7 +84,7 @@ TEST(ScudoSecondaryTest, SecondaryIterat
   L->init(nullptr);
   std::vector<void *> V;
   const scudo::uptr PageSize = scudo::getPageSizeCached();
-  for (scudo::u8 I = 0; I < 32; I++)
+  for (scudo::uptr I = 0; I < 32U; I++)
     V.push_back(L->allocate((std::rand() % 16) * PageSize));
   auto Lambda = [V](scudo::uptr Block) {
     EXPECT_NE(std::find(V.begin(), V.end(), reinterpret_cast<void *>(Block)),
@@ -100,19 +100,19 @@ TEST(ScudoSecondaryTest, SecondaryIterat
   L->printStats();
 }
 
-std::mutex Mutex;
-std::condition_variable Cv;
-bool Ready = false;
+static std::mutex Mutex;
+static std::condition_variable Cv;
+static bool Ready = false;
 
 static void performAllocations(scudo::MapAllocator *L) {
+  std::vector<void *> V;
+  const scudo::uptr PageSize = scudo::getPageSizeCached();
   {
     std::unique_lock<std::mutex> Lock(Mutex);
     while (!Ready)
       Cv.wait(Lock);
   }
-  std::vector<void *> V;
-  const scudo::uptr PageSize = scudo::getPageSizeCached();
-  for (scudo::u8 I = 0; I < 32; I++)
+  for (scudo::uptr I = 0; I < 32U; I++)
     V.push_back(L->allocate((std::rand() % 16) * PageSize));
   while (!V.empty()) {
     L->deallocate(V.back());
@@ -124,7 +124,7 @@ TEST(ScudoSecondaryTest, SecondaryThread
   scudo::MapAllocator *L = new scudo::MapAllocator;
   L->init(nullptr);
   std::thread Threads[10];
-  for (scudo::uptr I = 0; I < 10; I++)
+  for (scudo::uptr I = 0; I < 10U; I++)
     Threads[I] = std::thread(performAllocations, L);
   {
     std::unique_lock<std::mutex> Lock(Mutex);




More information about the llvm-commits mailing list