From 25ff296c4c8d35c63255531a86f877f20eeaeb62 Mon Sep 17 00:00:00 2001 From: hablich Date: Wed, 27 Apr 2016 05:39:31 -0700 Subject: [PATCH] Revert of [heap] Uncommit pooled pages concurrently (patchset #5 id:120001 of https://codereview.chromium.org/1913083002/ ) Reason for revert: Creates a spike of OOM(v8) crashers on Win32. Original issue's description: > [heap] Uncommit pooled pages concurrently > > - Move the concurrent unmapping to MemoryAllocator > - Hide (private) members where possible > - MemoryAllocator:Free is now the bottleneck for freeing > - Pooled pages are either allocated from a set of pooled pages are obtained > through work stealing from the concurrent unmapper > > BUG=chromium:605866, chromium:581412 > LOG=N > > Committed: https://crrev.com/2158df87116906160cebc3ad20c97f454822da03 > Cr-Commit-Position: refs/heads/master@{#35797} TBR=hpayer@chromium.org,mlippautz@chromium.org # Skipping CQ checks because original CL landed less than 1 days ago. NOPRESUBMIT=true NOTREECHECKS=true NOTRY=true BUG=chromium:605866, chromium:581412 Review URL: https://codereview.chromium.org/1925563003 Cr-Commit-Position: refs/heads/master@{#35819} --- src/deoptimizer.cc | 2 +- src/heap/heap.cc | 74 ++++++++++++++++ src/heap/heap.h | 12 +++ src/heap/mark-compact.cc | 6 +- src/heap/spaces.cc | 149 +++++++++----------------------- src/heap/spaces.h | 114 ++++-------------------- test/cctest/heap/test-spaces.cc | 2 +- 7 files changed, 149 insertions(+), 210 deletions(-) diff --git a/src/deoptimizer.cc b/src/deoptimizer.cc index 3e699922de9..bf7cd935f6e 100644 --- a/src/deoptimizer.cc +++ b/src/deoptimizer.cc @@ -47,7 +47,7 @@ DeoptimizerData::DeoptimizerData(MemoryAllocator* allocator) DeoptimizerData::~DeoptimizerData() { for (int i = 0; i <= Deoptimizer::kLastBailoutType; ++i) { - allocator_->Free(deopt_entry_code_[i]); + allocator_->Free(deopt_entry_code_[i]); deopt_entry_code_[i] = NULL; } } diff --git a/src/heap/heap.cc b/src/heap/heap.cc index dbda987b449..d95da5e0225 100644 --- a/src/heap/heap.cc +++ b/src/heap/heap.cc @@ -157,6 +157,9 @@ Heap::Heap() current_gc_flags_(Heap::kNoGCFlags), current_gc_callback_flags_(GCCallbackFlags::kNoGCCallbackFlags), external_string_table_(this), + chunks_queued_for_free_(NULL), + concurrent_unmapping_tasks_active_(0), + pending_unmapping_tasks_semaphore_(0), gc_callbacks_depth_(0), deserialization_complete_(false), strong_roots_list_(NULL), @@ -5444,6 +5447,8 @@ void Heap::TearDown() { delete scavenge_job_; scavenge_job_ = nullptr; + WaitUntilUnmappingOfFreeChunksCompleted(); + delete array_buffer_tracker_; array_buffer_tracker_ = nullptr; @@ -6249,6 +6254,75 @@ void Heap::ExternalStringTable::TearDown() { } +class Heap::UnmapFreeMemoryTask : public v8::Task { + public: + UnmapFreeMemoryTask(Heap* heap, MemoryChunk* head) + : heap_(heap), head_(head) {} + virtual ~UnmapFreeMemoryTask() {} + + private: + // v8::Task overrides. + void Run() override { + heap_->FreeQueuedChunks(head_); + heap_->pending_unmapping_tasks_semaphore_.Signal(); + } + + Heap* heap_; + MemoryChunk* head_; + + DISALLOW_COPY_AND_ASSIGN(UnmapFreeMemoryTask); +}; + + +void Heap::WaitUntilUnmappingOfFreeChunksCompleted() { + while (concurrent_unmapping_tasks_active_ > 0) { + pending_unmapping_tasks_semaphore_.Wait(); + concurrent_unmapping_tasks_active_--; + } +} + + +void Heap::QueueMemoryChunkForFree(MemoryChunk* chunk) { + // PreFree logically frees the memory chunk. However, the actual freeing + // will happen on a separate thread sometime later. + memory_allocator()->PreFreeMemory(chunk); + + // The chunks added to this queue will be freed by a concurrent thread. + chunk->set_next_chunk(chunks_queued_for_free_); + chunks_queued_for_free_ = chunk; +} + + +void Heap::FreeQueuedChunks() { + if (chunks_queued_for_free_ != NULL) { + if (FLAG_concurrent_sweeping) { + V8::GetCurrentPlatform()->CallOnBackgroundThread( + new UnmapFreeMemoryTask(this, chunks_queued_for_free_), + v8::Platform::kShortRunningTask); + } else { + FreeQueuedChunks(chunks_queued_for_free_); + pending_unmapping_tasks_semaphore_.Signal(); + } + chunks_queued_for_free_ = NULL; + } else { + // If we do not have anything to unmap, we just signal the semaphore + // that we are done. + pending_unmapping_tasks_semaphore_.Signal(); + } + concurrent_unmapping_tasks_active_++; +} + + +void Heap::FreeQueuedChunks(MemoryChunk* list_head) { + MemoryChunk* next; + MemoryChunk* chunk; + for (chunk = list_head; chunk != NULL; chunk = next) { + next = chunk->next_chunk(); + memory_allocator()->PerformFreeMemory(chunk); + } +} + + void Heap::RememberUnmappedPage(Address page, bool compacted) { uintptr_t p = reinterpret_cast(page); // Tag the page pointer to make it findable in the dump file. diff --git a/src/heap/heap.h b/src/heap/heap.h index b29eeb44bc1..2628898c862 100644 --- a/src/heap/heap.h +++ b/src/heap/heap.h @@ -778,6 +778,11 @@ class Heap { inline bool OldGenerationAllocationLimitReached(); + void QueueMemoryChunkForFree(MemoryChunk* chunk); + void FreeQueuedChunks(MemoryChunk* list_head); + void FreeQueuedChunks(); + void WaitUntilUnmappingOfFreeChunksCompleted(); + // Completely clear the Instanceof cache (to stop it keeping objects alive // around a GC). inline void CompletelyClearInstanceofCache(); @@ -1385,6 +1390,7 @@ class Heap { private: class PretenuringScope; + class UnmapFreeMemoryTask; // External strings table is a place where all external strings are // registered. We need to keep track of such strings to properly @@ -2202,6 +2208,12 @@ class Heap { ExternalStringTable external_string_table_; + MemoryChunk* chunks_queued_for_free_; + + size_t concurrent_unmapping_tasks_active_; + + base::Semaphore pending_unmapping_tasks_semaphore_; + base::Mutex relocation_mutex_; int gc_callbacks_depth_; diff --git a/src/heap/mark-compact.cc b/src/heap/mark-compact.cc index 4132d0b90ca..96940ed85f2 100644 --- a/src/heap/mark-compact.cc +++ b/src/heap/mark-compact.cc @@ -846,7 +846,7 @@ void MarkCompactCollector::Prepare() { // If concurrent unmapping tasks are still running, we should wait for // them here. - heap()->memory_allocator()->unmapper()->WaitUntilCompleted(); + heap()->WaitUntilUnmappingOfFreeChunksCompleted(); // Clear marking bits if incremental marking is aborted. if (was_marked_incrementally_ && heap_->ShouldAbortIncrementalMarking()) { @@ -3539,7 +3539,7 @@ void MarkCompactCollector::EvacuateNewSpaceAndCandidates() { // slots only handles old space (for unboxed doubles), and thus map space can // still contain stale pointers. We only free the chunks after pointer updates // to still have access to page headers. - heap()->memory_allocator()->unmapper()->FreeQueuedChunks(); + heap()->FreeQueuedChunks(); { TRACE_GC(heap()->tracer(), GCTracer::Scope::MC_EVACUATE_CLEAN_UP); @@ -3727,7 +3727,7 @@ void MarkCompactCollector::ReleaseEvacuationCandidates() { } evacuation_candidates_.Rewind(0); compacting_ = false; - heap()->memory_allocator()->unmapper()->FreeQueuedChunks(); + heap()->FreeQueuedChunks(); } int MarkCompactCollector::Sweeper::ParallelSweepSpace(AllocationSpace identity, diff --git a/src/heap/spaces.cc b/src/heap/spaces.cc index d750440d5f5..98c0c4a3661 100644 --- a/src/heap/spaces.cc +++ b/src/heap/spaces.cc @@ -6,13 +6,11 @@ #include "src/base/bits.h" #include "src/base/platform/platform.h" -#include "src/base/platform/semaphore.h" #include "src/full-codegen/full-codegen.h" #include "src/heap/slot-set.h" #include "src/macro-assembler.h" #include "src/msan.h" #include "src/snapshot/snapshot.h" -#include "src/v8.h" namespace v8 { namespace internal { @@ -305,8 +303,7 @@ MemoryAllocator::MemoryAllocator(Isolate* isolate) size_(0), size_executable_(0), lowest_ever_allocated_(reinterpret_cast(-1)), - highest_ever_allocated_(reinterpret_cast(0)), - unmapper_(this) {} + highest_ever_allocated_(reinterpret_cast(0)) {} bool MemoryAllocator::SetUp(intptr_t capacity, intptr_t capacity_executable, intptr_t code_range_size) { @@ -325,14 +322,10 @@ bool MemoryAllocator::SetUp(intptr_t capacity, intptr_t capacity_executable, void MemoryAllocator::TearDown() { - unmapper()->WaitUntilCompleted(); - - MemoryChunk* chunk = nullptr; - while ((chunk = unmapper()->TryGetPooledMemoryChunkSafe()) != nullptr) { + for (MemoryChunk* chunk : chunk_pool_) { FreeMemory(reinterpret_cast
(chunk), MemoryChunk::kPageSize, NOT_EXECUTABLE); } - // Check that spaces were torn down before MemoryAllocator. DCHECK_EQ(size_.Value(), 0); // TODO(gc) this will be true again when we fix FreeMemory. @@ -348,55 +341,6 @@ void MemoryAllocator::TearDown() { code_range_ = nullptr; } -class MemoryAllocator::Unmapper::UnmapFreeMemoryTask : public v8::Task { - public: - explicit UnmapFreeMemoryTask(Unmapper* unmapper) : unmapper_(unmapper) {} - - private: - // v8::Task overrides. - void Run() override { - unmapper_->PerformFreeMemoryOnQueuedChunks(); - unmapper_->pending_unmapping_tasks_semaphore_.Signal(); - } - - Unmapper* unmapper_; - DISALLOW_COPY_AND_ASSIGN(UnmapFreeMemoryTask); -}; - -void MemoryAllocator::Unmapper::FreeQueuedChunks() { - if (FLAG_concurrent_sweeping) { - V8::GetCurrentPlatform()->CallOnBackgroundThread( - new UnmapFreeMemoryTask(this), v8::Platform::kShortRunningTask); - concurrent_unmapping_tasks_active_++; - } else { - PerformFreeMemoryOnQueuedChunks(); - } -} - -bool MemoryAllocator::Unmapper::WaitUntilCompleted() { - bool waited = false; - while (concurrent_unmapping_tasks_active_ > 0) { - pending_unmapping_tasks_semaphore_.Wait(); - concurrent_unmapping_tasks_active_--; - waited = true; - } - return waited; -} - -void MemoryAllocator::Unmapper::PerformFreeMemoryOnQueuedChunks() { - MemoryChunk* chunk = nullptr; - // Regular chunks. - while ((chunk = GetMemoryChunkSafe()) != nullptr) { - bool pooled = chunk->IsFlagSet(MemoryChunk::POOLED); - allocator_->PerformFreeMemory(chunk); - if (pooled) AddMemoryChunkSafe(chunk); - } - // Non-regular chunks. - while ((chunk = GetMemoryChunkSafe()) != nullptr) { - allocator_->PerformFreeMemory(chunk); - } -} - bool MemoryAllocator::CommitMemory(Address base, size_t size, Executability executable) { if (!base::VirtualMemory::CommitRegion(base, size, @@ -796,45 +740,28 @@ void MemoryAllocator::PerformFreeMemory(MemoryChunk* chunk) { chunk->ReleaseAllocatedMemory(); base::VirtualMemory* reservation = chunk->reserved_memory(); - if (chunk->IsFlagSet(MemoryChunk::POOLED)) { - UncommitBlock(reinterpret_cast
(chunk), MemoryChunk::kPageSize); + if (reservation->IsReserved()) { + FreeMemory(reservation, chunk->executable()); } else { - if (reservation->IsReserved()) { - FreeMemory(reservation, chunk->executable()); - } else { - FreeMemory(chunk->address(), chunk->size(), chunk->executable()); - } + FreeMemory(chunk->address(), chunk->size(), chunk->executable()); } } -template +template void MemoryAllocator::Free(MemoryChunk* chunk) { - switch (mode) { - case kFull: - PreFreeMemory(chunk); - PerformFreeMemory(chunk); - break; - case kPooledAndQueue: - DCHECK_EQ(chunk->size(), static_cast(MemoryChunk::kPageSize)); - DCHECK_EQ(chunk->executable(), NOT_EXECUTABLE); - chunk->SetFlag(MemoryChunk::POOLED); - // Fall through to kPreFreeAndQueue. - case kPreFreeAndQueue: - PreFreeMemory(chunk); - // The chunks added to this queue will be freed by a concurrent thread. - unmapper()->AddMemoryChunkSafe(chunk); - break; - default: - UNREACHABLE(); + if (mode == kRegular) { + PreFreeMemory(chunk); + PerformFreeMemory(chunk); + } else { + DCHECK_EQ(mode, kPooled); + FreePooled(chunk); } } -template void MemoryAllocator::Free(MemoryChunk* chunk); - -template void MemoryAllocator::Free( +template void MemoryAllocator::Free( MemoryChunk* chunk); -template void MemoryAllocator::Free( +template void MemoryAllocator::Free( MemoryChunk* chunk); template @@ -873,9 +800,9 @@ LargePage* MemoryAllocator::AllocateLargePage(intptr_t size, template MemoryChunk* MemoryAllocator::AllocatePagePooled(SpaceType* owner) { - MemoryChunk* chunk = unmapper()->TryGetPooledMemoryChunkSafe(); - if (chunk == nullptr) return nullptr; + if (chunk_pool_.is_empty()) return nullptr; const int size = MemoryChunk::kPageSize; + MemoryChunk* chunk = chunk_pool_.RemoveLast(); const Address start = reinterpret_cast
(chunk); const Address area_start = start + MemoryChunk::kObjectStartOffset; const Address area_end = start + size; @@ -887,6 +814,18 @@ MemoryChunk* MemoryAllocator::AllocatePagePooled(SpaceType* owner) { return chunk; } +void MemoryAllocator::FreePooled(MemoryChunk* chunk) { + DCHECK_EQ(chunk->size(), static_cast(MemoryChunk::kPageSize)); + DCHECK_EQ(chunk->executable(), NOT_EXECUTABLE); + chunk_pool_.Add(chunk); + intptr_t chunk_size = static_cast(chunk->size()); + if (chunk->executable() == EXECUTABLE) { + size_executable_.Increment(-chunk_size); + } + size_.Increment(-chunk_size); + UncommitBlock(reinterpret_cast
(chunk), MemoryChunk::kPageSize); +} + bool MemoryAllocator::CommitBlock(Address start, size_t size, Executability executable) { if (!CommitMemory(start, size, executable)) return false; @@ -1029,16 +968,12 @@ bool MemoryAllocator::CommitExecutableMemory(base::VirtualMemory* vm, // MemoryChunk implementation void MemoryChunk::ReleaseAllocatedMemory() { - if (skip_list_ != nullptr) { - delete skip_list_; - skip_list_ = nullptr; - } - if (mutex_ != nullptr) { - delete mutex_; - mutex_ = nullptr; - } - if (old_to_new_slots_ != nullptr) ReleaseOldToNewSlots(); - if (old_to_old_slots_ != nullptr) ReleaseOldToOldSlots(); + delete skip_list_; + skip_list_ = nullptr; + delete mutex_; + mutex_ = nullptr; + ReleaseOldToNewSlots(); + ReleaseOldToOldSlots(); } static SlotSet* AllocateSlotSet(size_t size, Address page_start) { @@ -1120,7 +1055,7 @@ bool PagedSpace::HasBeenSetUp() { return true; } void PagedSpace::TearDown() { PageIterator iterator(this); while (iterator.has_next()) { - heap()->memory_allocator()->Free(iterator.next()); + heap()->memory_allocator()->Free(iterator.next()); } anchor_.set_next_page(&anchor_); anchor_.set_prev_page(&anchor_); @@ -1310,7 +1245,7 @@ void PagedSpace::ReleasePage(Page* page) { } AccountUncommitted(static_cast(page->size())); - heap()->memory_allocator()->Free(page); + heap()->QueueMemoryChunkForFree(page); DCHECK(Capacity() > 0); accounting_stats_.ShrinkSpace(AreaSize()); @@ -1772,14 +1707,12 @@ bool SemiSpace::Uncommit() { DCHECK(is_committed()); NewSpacePageIterator it(this); while (it.has_next()) { - heap()->memory_allocator()->Free( - it.next()); + heap()->memory_allocator()->Free(it.next()); } anchor()->set_next_page(anchor()); anchor()->set_prev_page(anchor()); AccountUncommitted(current_capacity_); committed_ = false; - heap()->memory_allocator()->unmapper()->FreeQueuedChunks(); return true; } @@ -1854,12 +1787,10 @@ bool SemiSpace::ShrinkTo(int new_capacity) { new_last_page = last_page->prev_page(); new_last_page->set_next_page(anchor()); anchor()->set_prev_page(new_last_page); - heap()->memory_allocator()->Free( - last_page); + heap()->memory_allocator()->Free(last_page); delta_pages--; } AccountUncommitted(static_cast(delta)); - heap()->memory_allocator()->unmapper()->FreeQueuedChunks(); } current_capacity_ = new_capacity; return true; @@ -2950,7 +2881,7 @@ void LargeObjectSpace::TearDown() { ObjectSpace space = static_cast(1 << identity()); heap()->memory_allocator()->PerformAllocationCallback( space, kAllocationActionFree, page->size()); - heap()->memory_allocator()->Free(page); + heap()->memory_allocator()->Free(page); } SetUp(); } @@ -3093,7 +3024,7 @@ void LargeObjectSpace::FreeUnmarkedObjects() { static_cast(key)); } - heap()->memory_allocator()->Free(page); + heap()->QueueMemoryChunkForFree(page); } } } diff --git a/src/heap/spaces.h b/src/heap/spaces.h index bdeba3b5b0f..c468ca4d87f 100644 --- a/src/heap/spaces.h +++ b/src/heap/spaces.h @@ -5,8 +5,6 @@ #ifndef V8_HEAP_SPACES_H_ #define V8_HEAP_SPACES_H_ -#include - #include "src/allocation.h" #include "src/atomic-utils.h" #include "src/base/atomicops.h" @@ -442,10 +440,6 @@ class MemoryChunk { // still has to be performed. PRE_FREED, - // |POOLED|: When actually freeing this chunk, only uncommit and do not - // give up the reservation as we still reuse the chunk at some point. - POOLED, - // |COMPACTION_WAS_ABORTED|: Indicates that the compaction in this page // has been aborted and needs special handling by the sweeper. COMPACTION_WAS_ABORTED, @@ -1271,92 +1265,15 @@ class SkipList { // A space acquires chunks of memory from the operating system. The memory // allocator allocated and deallocates pages for the paged heap spaces and large // pages for large object space. +// +// Each space has to manage it's own pages. +// class MemoryAllocator { public: - // Unmapper takes care of concurrently unmapping and uncommitting memory - // chunks. - class Unmapper { - public: - class UnmapFreeMemoryTask; - - explicit Unmapper(MemoryAllocator* allocator) - : allocator_(allocator), - pending_unmapping_tasks_semaphore_(0), - concurrent_unmapping_tasks_active_(0) {} - - void AddMemoryChunkSafe(MemoryChunk* chunk) { - if ((chunk->size() == Page::kPageSize) && - (chunk->executable() == EXECUTABLE)) { - AddMemoryChunkSafe(chunk); - } else { - AddMemoryChunkSafe(chunk); - } - } - - MemoryChunk* TryGetPooledMemoryChunkSafe() { - // Procedure: - // (1) Try to get a chunk that was declared as pooled and already has - // been uncommitted. - // (2) Try to steal any memory chunk of kPageSize that would've been - // unmapped. - MemoryChunk* chunk = GetMemoryChunkSafe(); - if (chunk == nullptr) { - chunk = GetMemoryChunkSafe(); - if (chunk != nullptr) { - // For stolen chunks we need to manually free any allocated memory. - chunk->ReleaseAllocatedMemory(); - } - } - return chunk; - } - - void FreeQueuedChunks(); - bool WaitUntilCompleted(); - - private: - enum ChunkQueueType { - kRegular, // Pages of kPageSize that do not live in a CodeRange and - // can thus be used for stealing. - kNonRegular, // Large chunks and executable chunks. - kPooled, // Pooled chunks, already uncommited and ready for reuse. - kNumberOfChunkQueues, - }; - - template - void AddMemoryChunkSafe(MemoryChunk* chunk) { - base::LockGuard guard(&mutex_); - chunks_[type].push_back(chunk); - } - - template - MemoryChunk* GetMemoryChunkSafe() { - base::LockGuard guard(&mutex_); - if (chunks_[type].empty()) return nullptr; - MemoryChunk* chunk = chunks_[type].front(); - chunks_[type].pop_front(); - return chunk; - } - - void PerformFreeMemoryOnQueuedChunks(); - - base::Mutex mutex_; - MemoryAllocator* allocator_; - std::list chunks_[kNumberOfChunkQueues]; - base::Semaphore pending_unmapping_tasks_semaphore_; - intptr_t concurrent_unmapping_tasks_active_; - - friend class MemoryAllocator; - }; - enum AllocationMode { kRegular, kPooled, }; - enum FreeMode { - kFull, - kPreFreeAndQueue, - kPooledAndQueue, - }; explicit MemoryAllocator(Isolate* isolate); @@ -1377,7 +1294,16 @@ class MemoryAllocator { LargePage* AllocateLargePage(intptr_t size, LargeObjectSpace* owner, Executability executable); - template + // PreFree logically frees the object, i.e., it takes care of the size + // bookkeeping and calls the allocation callback. + void PreFreeMemory(MemoryChunk* chunk); + + // FreeMemory can be called concurrently when PreFree was executed before. + void PerformFreeMemory(MemoryChunk* chunk); + + // Free is a wrapper method. For kRegular AllocationMode it calls PreFree and + // PerformFreeMemory together. For kPooled it will dispatch to pooled free. + template void Free(MemoryChunk* chunk); // Returns allocated spaces in bytes. @@ -1483,21 +1409,16 @@ class MemoryAllocator { size_t reserved_size); CodeRange* code_range() { return code_range_; } - Unmapper* unmapper() { return &unmapper_; } private: - // PreFree logically frees the object, i.e., it takes care of the size - // bookkeeping and calls the allocation callback. - void PreFreeMemory(MemoryChunk* chunk); - - // FreeMemory can be called concurrently when PreFree was executed before. - void PerformFreeMemory(MemoryChunk* chunk); - // See AllocatePage for public interface. Note that currently we only support // pools for NOT_EXECUTABLE pages of size MemoryChunk::kPageSize. template MemoryChunk* AllocatePagePooled(SpaceType* owner); + // Free that chunk into the pool. + void FreePooled(MemoryChunk* chunk); + Isolate* isolate_; CodeRange* code_range_; @@ -1553,8 +1474,9 @@ class MemoryAllocator { } while ((high > ptr) && !highest_ever_allocated_.TrySetValue(ptr, high)); } + List chunk_pool_; + base::VirtualMemory last_chunk_; - Unmapper unmapper_; friend class TestCodeRangeScope; diff --git a/test/cctest/heap/test-spaces.cc b/test/cctest/heap/test-spaces.cc index da364c51408..1ed07c54a58 100644 --- a/test/cctest/heap/test-spaces.cc +++ b/test/cctest/heap/test-spaces.cc @@ -193,7 +193,7 @@ static void VerifyMemoryChunk(Isolate* isolate, CHECK(static_cast(memory_chunk->area_size()) == second_commit_area_size); - memory_allocator->Free(memory_chunk); + memory_allocator->Free(memory_chunk); } memory_allocator->TearDown(); delete memory_allocator;