summaryrefslogtreecommitdiff
path: root/deps/v8/src/heap
diff options
context:
space:
mode:
Diffstat (limited to 'deps/v8/src/heap')
-rw-r--r--deps/v8/src/heap/array-buffer-collector.cc64
-rw-r--r--deps/v8/src/heap/array-buffer-collector.h23
-rw-r--r--deps/v8/src/heap/array-buffer-tracker-inl.h9
-rw-r--r--deps/v8/src/heap/array-buffer-tracker.cc28
-rw-r--r--deps/v8/src/heap/array-buffer-tracker.h4
-rw-r--r--deps/v8/src/heap/concurrent-marking.cc121
-rw-r--r--deps/v8/src/heap/concurrent-marking.h5
-rw-r--r--deps/v8/src/heap/embedder-tracing.cc7
-rw-r--r--deps/v8/src/heap/embedder-tracing.h3
-rw-r--r--deps/v8/src/heap/factory-inl.h67
-rw-r--r--deps/v8/src/heap/factory.cc182
-rw-r--r--deps/v8/src/heap/factory.h58
-rw-r--r--deps/v8/src/heap/gc-tracer.cc7
-rw-r--r--deps/v8/src/heap/gc-tracer.h3
-rw-r--r--deps/v8/src/heap/heap-controller.cc82
-rw-r--r--deps/v8/src/heap/heap-controller.h47
-rw-r--r--deps/v8/src/heap/heap-inl.h60
-rw-r--r--deps/v8/src/heap/heap-write-barrier-inl.h4
-rw-r--r--deps/v8/src/heap/heap.cc703
-rw-r--r--deps/v8/src/heap/heap.h416
-rw-r--r--deps/v8/src/heap/incremental-marking-inl.h21
-rw-r--r--deps/v8/src/heap/incremental-marking-job.cc5
-rw-r--r--deps/v8/src/heap/incremental-marking.cc47
-rw-r--r--deps/v8/src/heap/incremental-marking.h21
-rw-r--r--deps/v8/src/heap/item-parallel-job.cc2
-rw-r--r--deps/v8/src/heap/item-parallel-job.h4
-rw-r--r--deps/v8/src/heap/mark-compact-inl.h98
-rw-r--r--deps/v8/src/heap/mark-compact.cc136
-rw-r--r--deps/v8/src/heap/mark-compact.h62
-rw-r--r--deps/v8/src/heap/marking.cc52
-rw-r--r--deps/v8/src/heap/object-stats.cc22
-rw-r--r--deps/v8/src/heap/objects-visiting-inl.h9
-rw-r--r--deps/v8/src/heap/objects-visiting.h6
-rw-r--r--deps/v8/src/heap/scavenge-job.cc5
-rw-r--r--deps/v8/src/heap/scavenger-inl.h279
-rw-r--r--deps/v8/src/heap/scavenger.cc286
-rw-r--r--deps/v8/src/heap/scavenger.h142
-rw-r--r--deps/v8/src/heap/setup-heap-internal.cc98
-rw-r--r--deps/v8/src/heap/spaces-inl.h45
-rw-r--r--deps/v8/src/heap/spaces.cc516
-rw-r--r--deps/v8/src/heap/spaces.h295
-rw-r--r--deps/v8/src/heap/store-buffer.cc8
-rw-r--r--deps/v8/src/heap/store-buffer.h2
-rw-r--r--deps/v8/src/heap/sweeper.cc13
44 files changed, 1968 insertions, 2099 deletions
diff --git a/deps/v8/src/heap/array-buffer-collector.cc b/deps/v8/src/heap/array-buffer-collector.cc
index 2c28f46a85..0cf4ae945d 100644
--- a/deps/v8/src/heap/array-buffer-collector.cc
+++ b/deps/v8/src/heap/array-buffer-collector.cc
@@ -12,50 +12,54 @@
namespace v8 {
namespace internal {
-void ArrayBufferCollector::AddGarbageAllocations(
+namespace {
+
+void FreeAllocationsHelper(
+ Heap* heap, const std::vector<JSArrayBuffer::Allocation>& allocations) {
+ for (JSArrayBuffer::Allocation alloc : allocations) {
+ JSArrayBuffer::FreeBackingStore(heap->isolate(), alloc);
+ }
+}
+
+} // namespace
+
+void ArrayBufferCollector::QueueOrFreeGarbageAllocations(
std::vector<JSArrayBuffer::Allocation> allocations) {
- base::LockGuard<base::Mutex> guard(&allocations_mutex_);
- allocations_.push_back(std::move(allocations));
+ if (heap_->ShouldReduceMemory()) {
+ FreeAllocationsHelper(heap_, allocations);
+ } else {
+ base::LockGuard<base::Mutex> guard(&allocations_mutex_);
+ allocations_.push_back(std::move(allocations));
+ }
}
-void ArrayBufferCollector::FreeAllocations() {
+void ArrayBufferCollector::PerformFreeAllocations() {
base::LockGuard<base::Mutex> guard(&allocations_mutex_);
for (const std::vector<JSArrayBuffer::Allocation>& allocations :
allocations_) {
- for (JSArrayBuffer::Allocation alloc : allocations) {
- JSArrayBuffer::FreeBackingStore(heap_->isolate(), alloc);
- }
+ FreeAllocationsHelper(heap_, allocations);
}
allocations_.clear();
}
-class ArrayBufferCollector::FreeingTask final : public CancelableTask {
- public:
- explicit FreeingTask(Heap* heap)
- : CancelableTask(heap->isolate()), heap_(heap) {}
-
- virtual ~FreeingTask() {}
-
- private:
- void RunInternal() final {
- TRACE_BACKGROUND_GC(
- heap_->tracer(),
- GCTracer::BackgroundScope::BACKGROUND_ARRAY_BUFFER_FREE);
- heap_->array_buffer_collector()->FreeAllocations();
- }
-
- Heap* heap_;
-};
-
-void ArrayBufferCollector::FreeAllocationsOnBackgroundThread() {
+void ArrayBufferCollector::FreeAllocations() {
// TODO(wez): Remove backing-store from external memory accounting.
heap_->account_external_memory_concurrently_freed();
- if (!heap_->IsTearingDown() && FLAG_concurrent_array_buffer_freeing) {
+ if (!heap_->IsTearingDown() && !heap_->ShouldReduceMemory() &&
+ FLAG_concurrent_array_buffer_freeing) {
V8::GetCurrentPlatform()->CallOnWorkerThread(
- base::make_unique<FreeingTask>(heap_));
+ MakeCancelableLambdaTask(heap_->isolate(), [this] {
+ TRACE_BACKGROUND_GC(
+ heap_->tracer(),
+ GCTracer::BackgroundScope::BACKGROUND_ARRAY_BUFFER_FREE);
+ PerformFreeAllocations();
+ }));
} else {
- // Fallback for when concurrency is disabled/restricted.
- FreeAllocations();
+ // Fallback for when concurrency is disabled/restricted. This is e.g. the
+ // case when the GC should reduce memory. For such GCs the
+ // QueueOrFreeGarbageAllocations() call would immediately free the
+ // allocations and this call would free already queued ones.
+ PerformFreeAllocations();
}
}
diff --git a/deps/v8/src/heap/array-buffer-collector.h b/deps/v8/src/heap/array-buffer-collector.h
index 74a28c3d06..784092e936 100644
--- a/deps/v8/src/heap/array-buffer-collector.h
+++ b/deps/v8/src/heap/array-buffer-collector.h
@@ -23,24 +23,27 @@ class ArrayBufferCollector {
public:
explicit ArrayBufferCollector(Heap* heap) : heap_(heap) {}
- ~ArrayBufferCollector() { FreeAllocations(); }
-
- // These allocations will begin to be freed once FreeAllocations() is called,
- // or on TearDown.
- void AddGarbageAllocations(
+ ~ArrayBufferCollector() { PerformFreeAllocations(); }
+
+ // These allocations will be either
+ // - freed immediately when under memory pressure, or
+ // - queued for freeing in FreeAllocations() or during tear down.
+ //
+ // FreeAllocations() potentially triggers a background task for processing.
+ void QueueOrFreeGarbageAllocations(
std::vector<JSArrayBuffer::Allocation> allocations);
// Calls FreeAllocations() on a background thread.
- void FreeAllocationsOnBackgroundThread();
+ void FreeAllocations();
private:
class FreeingTask;
- // Begin freeing the allocations added through AddGarbageAllocations. Also
- // called by TearDown.
- void FreeAllocations();
+ // Begin freeing the allocations added through QueueOrFreeGarbageAllocations.
+ // Also called by TearDown.
+ void PerformFreeAllocations();
- Heap* heap_;
+ Heap* const heap_;
base::Mutex allocations_mutex_;
std::vector<std::vector<JSArrayBuffer::Allocation>> allocations_;
};
diff --git a/deps/v8/src/heap/array-buffer-tracker-inl.h b/deps/v8/src/heap/array-buffer-tracker-inl.h
index e0d862aed7..814cfce63a 100644
--- a/deps/v8/src/heap/array-buffer-tracker-inl.h
+++ b/deps/v8/src/heap/array-buffer-tracker-inl.h
@@ -18,7 +18,7 @@ namespace internal {
void ArrayBufferTracker::RegisterNew(Heap* heap, JSArrayBuffer* buffer) {
if (buffer->backing_store() == nullptr) return;
- const size_t length = NumberToSize(buffer->byte_length());
+ const size_t length = buffer->byte_length();
Page* page = Page::FromAddress(buffer->address());
{
base::LockGuard<base::Mutex> guard(page->mutex());
@@ -42,7 +42,7 @@ void ArrayBufferTracker::Unregister(Heap* heap, JSArrayBuffer* buffer) {
if (buffer->backing_store() == nullptr) return;
Page* page = Page::FromAddress(buffer->address());
- const size_t length = NumberToSize(buffer->byte_length());
+ const size_t length = buffer->byte_length();
{
base::LockGuard<base::Mutex> guard(page->mutex());
LocalArrayBufferTracker* tracker = page->local_tracker();
@@ -100,6 +100,11 @@ void LocalArrayBufferTracker::Add(JSArrayBuffer* buffer, size_t length) {
page_->IncrementExternalBackingStoreBytes(
ExternalBackingStoreType::kArrayBuffer, length);
+ AddInternal(buffer, length);
+}
+
+void LocalArrayBufferTracker::AddInternal(JSArrayBuffer* buffer,
+ size_t length) {
auto ret = array_buffers_.insert(
{buffer,
{buffer->backing_store(), length, buffer->backing_store(),
diff --git a/deps/v8/src/heap/array-buffer-tracker.cc b/deps/v8/src/heap/array-buffer-tracker.cc
index 0a158e3543..f35f2b3754 100644
--- a/deps/v8/src/heap/array-buffer-tracker.cc
+++ b/deps/v8/src/heap/array-buffer-tracker.cc
@@ -26,11 +26,10 @@ void LocalArrayBufferTracker::Process(Callback callback) {
JSArrayBuffer* new_buffer = nullptr;
JSArrayBuffer* old_buffer = nullptr;
size_t freed_memory = 0;
- size_t moved_memory = 0;
for (TrackingData::iterator it = array_buffers_.begin();
it != array_buffers_.end(); ++it) {
old_buffer = it->first;
- Page* old_page = Page::FromAddress(old_buffer->address());
+ DCHECK_EQ(page_, Page::FromAddress(old_buffer->address()));
const CallbackResult result = callback(old_buffer, &new_buffer);
if (result == kKeepEntry) {
kept_array_buffers.insert(*it);
@@ -49,26 +48,25 @@ void LocalArrayBufferTracker::Process(Callback callback) {
// We should decrement before adding to avoid potential overflows in
// the external memory counters.
DCHECK_EQ(it->first->is_wasm_memory(), it->second.is_wasm_memory);
- old_page->DecrementExternalBackingStoreBytes(
- ExternalBackingStoreType::kArrayBuffer, length);
- tracker->Add(new_buffer, length);
+ tracker->AddInternal(new_buffer, length);
+ MemoryChunk::MoveExternalBackingStoreBytes(
+ ExternalBackingStoreType::kArrayBuffer,
+ static_cast<MemoryChunk*>(page_),
+ static_cast<MemoryChunk*>(target_page), length);
}
- moved_memory += it->second.length;
-
} else if (result == kRemoveEntry) {
- const size_t length = it->second.length;
- freed_memory += length;
+ freed_memory += it->second.length;
// We pass backing_store() and stored length to the collector for freeing
// the backing store. Wasm allocations will go through their own tracker
// based on the backing store.
backing_stores_to_free.push_back(it->second);
- old_page->DecrementExternalBackingStoreBytes(
- ExternalBackingStoreType::kArrayBuffer, length);
} else {
UNREACHABLE();
}
}
- if (moved_memory || freed_memory) {
+ if (freed_memory) {
+ page_->DecrementExternalBackingStoreBytes(
+ ExternalBackingStoreType::kArrayBuffer, freed_memory);
// TODO(wez): Remove backing-store from external memory accounting.
page_->heap()->update_external_memory_concurrently_freed(
static_cast<intptr_t>(freed_memory));
@@ -76,9 +74,9 @@ void LocalArrayBufferTracker::Process(Callback callback) {
array_buffers_.swap(kept_array_buffers);
- // Pass the backing stores that need to be freed to the main thread for later
- // distribution.
- page_->heap()->array_buffer_collector()->AddGarbageAllocations(
+ // Pass the backing stores that need to be freed to the main thread for
+ // potential later distribution.
+ page_->heap()->array_buffer_collector()->QueueOrFreeGarbageAllocations(
std::move(backing_stores_to_free));
}
diff --git a/deps/v8/src/heap/array-buffer-tracker.h b/deps/v8/src/heap/array-buffer-tracker.h
index e60fe6c6c0..3c00c2c486 100644
--- a/deps/v8/src/heap/array-buffer-tracker.h
+++ b/deps/v8/src/heap/array-buffer-tracker.h
@@ -113,6 +113,10 @@ class LocalArrayBufferTracker {
typedef std::unordered_map<JSArrayBuffer*, JSArrayBuffer::Allocation, Hasher>
TrackingData;
+ // Internal version of add that does not update counters. Requires separate
+ // logic for updating external memory counters.
+ inline void AddInternal(JSArrayBuffer* buffer, size_t length);
+
inline Space* space();
Page* page_;
diff --git a/deps/v8/src/heap/concurrent-marking.cc b/deps/v8/src/heap/concurrent-marking.cc
index f6eabbb021..5e147ca9a5 100644
--- a/deps/v8/src/heap/concurrent-marking.cc
+++ b/deps/v8/src/heap/concurrent-marking.cc
@@ -74,15 +74,19 @@ class ConcurrentMarkingVisitor final
public:
using BaseClass = HeapVisitor<int, ConcurrentMarkingVisitor>;
- explicit ConcurrentMarkingVisitor(ConcurrentMarking::MarkingWorklist* shared,
- ConcurrentMarking::MarkingWorklist* bailout,
- LiveBytesMap* live_bytes,
- WeakObjects* weak_objects, int task_id)
+ explicit ConcurrentMarkingVisitor(
+ ConcurrentMarking::MarkingWorklist* shared,
+ ConcurrentMarking::MarkingWorklist* bailout, LiveBytesMap* live_bytes,
+ WeakObjects* weak_objects,
+ ConcurrentMarking::EmbedderTracingWorklist* embedder_objects, int task_id,
+ bool embedder_tracing_enabled)
: shared_(shared, task_id),
bailout_(bailout, task_id),
weak_objects_(weak_objects),
+ embedder_objects_(embedder_objects, task_id),
marking_state_(live_bytes),
- task_id_(task_id) {}
+ task_id_(task_id),
+ embedder_tracing_enabled_(embedder_tracing_enabled) {}
template <typename T>
static V8_INLINE T* Cast(HeapObject* object) {
@@ -138,19 +142,24 @@ class ConcurrentMarkingVisitor final
for (MaybeObject** slot = start; slot < end; slot++) {
MaybeObject* object = base::AsAtomicPointer::Relaxed_Load(slot);
HeapObject* heap_object;
- if (object->ToStrongHeapObject(&heap_object)) {
+ if (object->GetHeapObjectIfStrong(&heap_object)) {
// If the reference changes concurrently from strong to weak, the write
// barrier will treat the weak reference as strong, so we won't miss the
// weak reference.
ProcessStrongHeapObject(host, reinterpret_cast<Object**>(slot),
heap_object);
- } else if (object->ToWeakHeapObject(&heap_object)) {
+ } else if (object->GetHeapObjectIfWeak(&heap_object)) {
ProcessWeakHeapObject(
host, reinterpret_cast<HeapObjectReference**>(slot), heap_object);
}
}
}
+ // Weak list pointers should be ignored during marking. The lists are
+ // reconstructed after GC.
+ void VisitCustomWeakPointers(HeapObject* host, Object** start,
+ Object** end) override {}
+
void VisitPointersInSnapshot(HeapObject* host, const SlotSnapshot& snapshot) {
for (int i = 0; i < snapshot.number_of_slots(); i++) {
Object** slot = snapshot.slot(i);
@@ -175,31 +184,27 @@ class ConcurrentMarkingVisitor final
return VisitJSObjectSubclass(map, object);
}
- int VisitJSArrayBuffer(Map* map, JSArrayBuffer* object) {
- return VisitJSObjectSubclass(map, object);
- }
-
int VisitWasmInstanceObject(Map* map, WasmInstanceObject* object) {
return VisitJSObjectSubclass(map, object);
}
+ // Some JS objects can carry back links to embedders that contain information
+ // relevant to the garbage collectors.
+
int VisitJSApiObject(Map* map, JSObject* object) {
- if (marking_state_.IsGrey(object)) {
- // The main thread will do wrapper tracing in Blink.
- bailout_.Push(object);
- }
- return 0;
+ return VisitEmbedderTracingSubclass(map, object);
}
- int VisitJSFunction(Map* map, JSFunction* object) {
- int size = JSFunction::BodyDescriptorWeak::SizeOf(map, object);
- int used_size = map->UsedInstanceSize();
- DCHECK_LE(used_size, size);
- DCHECK_GE(used_size, JSObject::kHeaderSize);
- const SlotSnapshot& snapshot = MakeSlotSnapshotWeak(map, object, used_size);
- if (!ShouldVisit(object)) return 0;
- VisitPointersInSnapshot(object, snapshot);
- return size;
+ int VisitJSArrayBuffer(Map* map, JSArrayBuffer* object) {
+ return VisitEmbedderTracingSubclass(map, object);
+ }
+
+ int VisitJSDataView(Map* map, JSDataView* object) {
+ return VisitEmbedderTracingSubclass(map, object);
+ }
+
+ int VisitJSTypedArray(Map* map, JSTypedArray* object) {
+ return VisitEmbedderTracingSubclass(map, object);
}
// ===========================================================================
@@ -270,34 +275,18 @@ class ConcurrentMarkingVisitor final
}
// ===========================================================================
- // Objects with weak fields and/or side-effectiful visitation.
+ // Side-effectful visitation.
// ===========================================================================
int VisitBytecodeArray(Map* map, BytecodeArray* object) {
if (!ShouldVisit(object)) return 0;
- int size = BytecodeArray::BodyDescriptorWeak::SizeOf(map, object);
+ int size = BytecodeArray::BodyDescriptor::SizeOf(map, object);
VisitMapPointer(object, object->map_slot());
- BytecodeArray::BodyDescriptorWeak::IterateBody(map, object, size, this);
+ BytecodeArray::BodyDescriptor::IterateBody(map, object, size, this);
object->MakeOlder();
return size;
}
- int VisitAllocationSite(Map* map, AllocationSite* object) {
- if (!ShouldVisit(object)) return 0;
- int size = AllocationSite::BodyDescriptorWeak::SizeOf(map, object);
- VisitMapPointer(object, object->map_slot());
- AllocationSite::BodyDescriptorWeak::IterateBody(map, object, size, this);
- return size;
- }
-
- int VisitCodeDataContainer(Map* map, CodeDataContainer* object) {
- if (!ShouldVisit(object)) return 0;
- int size = CodeDataContainer::BodyDescriptorWeak::SizeOf(map, object);
- VisitMapPointer(object, object->map_slot());
- CodeDataContainer::BodyDescriptorWeak::IterateBody(map, object, size, this);
- return size;
- }
-
int VisitMap(Map* meta_map, Map* map) {
if (marking_state_.IsGrey(map)) {
// Maps have ad-hoc weakness for descriptor arrays. They also clear the
@@ -315,14 +304,6 @@ class ConcurrentMarkingVisitor final
return 0;
}
- int VisitNativeContext(Map* map, Context* object) {
- if (!ShouldVisit(object)) return 0;
- int size = Context::BodyDescriptorWeak::SizeOf(map, object);
- VisitMapPointer(object, object->map_slot());
- Context::BodyDescriptorWeak::IterateBody(map, object, size, this);
- return size;
- }
-
int VisitTransitionArray(Map* map, TransitionArray* array) {
if (!ShouldVisit(array)) return 0;
VisitMapPointer(array, array->map_slot());
@@ -443,6 +424,18 @@ class ConcurrentMarkingVisitor final
}
template <typename T>
+ int VisitEmbedderTracingSubclass(Map* map, T* object) {
+ DCHECK(object->IsApiWrapper());
+ int size = VisitJSObjectSubclass(map, object);
+ if (size && embedder_tracing_enabled_) {
+ // Success: The object needs to be processed for embedder references on
+ // the main thread.
+ embedder_objects_.Push(object);
+ }
+ return size;
+ }
+
+ template <typename T>
int VisitLeftTrimmableArray(Map* map, T* object) {
// The synchronized_length() function checks that the length is a Smi.
// This is not necessarily the case if the array is being left-trimmed.
@@ -466,20 +459,14 @@ class ConcurrentMarkingVisitor final
return slot_snapshot_;
}
- template <typename T>
- const SlotSnapshot& MakeSlotSnapshotWeak(Map* map, T* object, int size) {
- SlotSnapshottingVisitor visitor(&slot_snapshot_);
- visitor.VisitPointer(object,
- reinterpret_cast<Object**>(object->map_slot()));
- T::BodyDescriptorWeak::IterateBody(map, object, size, &visitor);
- return slot_snapshot_;
- }
ConcurrentMarking::MarkingWorklist::View shared_;
ConcurrentMarking::MarkingWorklist::View bailout_;
WeakObjects* weak_objects_;
+ ConcurrentMarking::EmbedderTracingWorklist::View embedder_objects_;
ConcurrentMarkingState marking_state_;
int task_id_;
SlotSnapshot slot_snapshot_;
+ bool embedder_tracing_enabled_;
};
// Strings can change maps due to conversion to thin string or external strings.
@@ -524,7 +511,7 @@ class ConcurrentMarking::Task : public CancelableTask {
task_state_(task_state),
task_id_(task_id) {}
- virtual ~Task() {}
+ ~Task() override = default;
private:
// v8::internal::CancelableTask overrides.
@@ -541,12 +528,14 @@ class ConcurrentMarking::Task : public CancelableTask {
ConcurrentMarking::ConcurrentMarking(Heap* heap, MarkingWorklist* shared,
MarkingWorklist* bailout,
MarkingWorklist* on_hold,
- WeakObjects* weak_objects)
+ WeakObjects* weak_objects,
+ EmbedderTracingWorklist* embedder_objects)
: heap_(heap),
shared_(shared),
bailout_(bailout),
on_hold_(on_hold),
- weak_objects_(weak_objects) {
+ weak_objects_(weak_objects),
+ embedder_objects_(embedder_objects) {
// The runtime flag should be set only if the compile time flag was set.
#ifndef V8_CONCURRENT_MARKING
CHECK(!FLAG_concurrent_marking);
@@ -558,8 +547,9 @@ void ConcurrentMarking::Run(int task_id, TaskState* task_state) {
GCTracer::BackgroundScope::MC_BACKGROUND_MARKING);
size_t kBytesUntilInterruptCheck = 64 * KB;
int kObjectsUntilInterrupCheck = 1000;
- ConcurrentMarkingVisitor visitor(shared_, bailout_, &task_state->live_bytes,
- weak_objects_, task_id);
+ ConcurrentMarkingVisitor visitor(
+ shared_, bailout_, &task_state->live_bytes, weak_objects_,
+ embedder_objects_, task_id, heap_->local_embedder_heap_tracer()->InUse());
double time_ms;
size_t marked_bytes = 0;
if (FLAG_trace_concurrent_marking) {
@@ -626,6 +616,7 @@ void ConcurrentMarking::Run(int task_id, TaskState* task_state) {
shared_->FlushToGlobal(task_id);
bailout_->FlushToGlobal(task_id);
on_hold_->FlushToGlobal(task_id);
+ embedder_objects_->FlushToGlobal(task_id);
weak_objects_->transition_arrays.FlushToGlobal(task_id);
weak_objects_->ephemeron_hash_tables.FlushToGlobal(task_id);
diff --git a/deps/v8/src/heap/concurrent-marking.h b/deps/v8/src/heap/concurrent-marking.h
index 0b8ffd9336..34de02fea1 100644
--- a/deps/v8/src/heap/concurrent-marking.h
+++ b/deps/v8/src/heap/concurrent-marking.h
@@ -58,10 +58,12 @@ class ConcurrentMarking {
// task 0, reserved for the main thread).
static constexpr int kMaxTasks = 7;
using MarkingWorklist = Worklist<HeapObject*, 64 /* segment size */>;
+ using EmbedderTracingWorklist = Worklist<HeapObject*, 16 /* segment size */>;
ConcurrentMarking(Heap* heap, MarkingWorklist* shared,
MarkingWorklist* bailout, MarkingWorklist* on_hold,
- WeakObjects* weak_objects);
+ WeakObjects* weak_objects,
+ EmbedderTracingWorklist* embedder_objects);
// Schedules asynchronous tasks to perform concurrent marking. Objects in the
// heap should not be moved while these are active (can be stopped safely via
@@ -108,6 +110,7 @@ class ConcurrentMarking {
MarkingWorklist* const bailout_;
MarkingWorklist* const on_hold_;
WeakObjects* const weak_objects_;
+ EmbedderTracingWorklist* const embedder_objects_;
TaskState task_state_[kMaxTasks + 1];
std::atomic<size_t> total_marked_bytes_{0};
std::atomic<bool> ephemeron_marked_{false};
diff --git a/deps/v8/src/heap/embedder-tracing.cc b/deps/v8/src/heap/embedder-tracing.cc
index bf6d5f3b90..198cdd4b1a 100644
--- a/deps/v8/src/heap/embedder-tracing.cc
+++ b/deps/v8/src/heap/embedder-tracing.cc
@@ -24,13 +24,6 @@ void LocalEmbedderHeapTracer::TraceEpilogue() {
remote_tracer_->TraceEpilogue();
}
-void LocalEmbedderHeapTracer::AbortTracing() {
- if (!InUse()) return;
-
- cached_wrappers_to_trace_.clear();
- remote_tracer_->AbortTracing();
-}
-
void LocalEmbedderHeapTracer::EnterFinalPause() {
if (!InUse()) return;
diff --git a/deps/v8/src/heap/embedder-tracing.h b/deps/v8/src/heap/embedder-tracing.h
index ab8a46bb53..2588200db9 100644
--- a/deps/v8/src/heap/embedder-tracing.h
+++ b/deps/v8/src/heap/embedder-tracing.h
@@ -24,6 +24,8 @@ class V8_EXPORT_PRIVATE LocalEmbedderHeapTracer final {
if (remote_tracer_) remote_tracer_->isolate_ = nullptr;
}
+ EmbedderHeapTracer* remote_tracer() const { return remote_tracer_; }
+
void SetRemoteTracer(EmbedderHeapTracer* tracer) {
if (remote_tracer_) remote_tracer_->isolate_ = nullptr;
@@ -36,7 +38,6 @@ class V8_EXPORT_PRIVATE LocalEmbedderHeapTracer final {
void TracePrologue();
void TraceEpilogue();
- void AbortTracing();
void EnterFinalPause();
bool Trace(double deadline);
bool IsRemoteTracingDone();
diff --git a/deps/v8/src/heap/factory-inl.h b/deps/v8/src/heap/factory-inl.h
index 614c6ec174..eb1661aaee 100644
--- a/deps/v8/src/heap/factory-inl.h
+++ b/deps/v8/src/heap/factory-inl.h
@@ -16,73 +16,14 @@
namespace v8 {
namespace internal {
-#define ROOT_ACCESSOR(type, name, camel_name) \
- Handle<type> Factory::name() { \
- return Handle<type>(bit_cast<type**>( \
- &isolate()->heap()->roots_[Heap::k##camel_name##RootIndex])); \
+#define ROOT_ACCESSOR(type, name, CamelName) \
+ Handle<type> Factory::name() { \
+ return Handle<type>(bit_cast<type**>( \
+ &isolate()->heap()->roots_[RootIndex::k##CamelName])); \
}
ROOT_LIST(ROOT_ACCESSOR)
#undef ROOT_ACCESSOR
-#define STRUCT_MAP_ACCESSOR(NAME, Name, name) \
- Handle<Map> Factory::name##_map() { \
- return Handle<Map>(bit_cast<Map**>( \
- &isolate()->heap()->roots_[Heap::k##Name##MapRootIndex])); \
- }
-STRUCT_LIST(STRUCT_MAP_ACCESSOR)
-#undef STRUCT_MAP_ACCESSOR
-
-#define ALLOCATION_SITE_MAP_ACCESSOR(NAME, Name, Size, name) \
- Handle<Map> Factory::name##_map() { \
- return Handle<Map>(bit_cast<Map**>( \
- &isolate()->heap()->roots_[Heap::k##Name##Size##MapRootIndex])); \
- }
-ALLOCATION_SITE_LIST(ALLOCATION_SITE_MAP_ACCESSOR)
-#undef ALLOCATION_SITE_MAP_ACCESSOR
-
-#define DATA_HANDLER_MAP_ACCESSOR(NAME, Name, Size, name) \
- Handle<Map> Factory::name##_map() { \
- return Handle<Map>(bit_cast<Map**>( \
- &isolate()->heap()->roots_[Heap::k##Name##Size##MapRootIndex])); \
- }
-DATA_HANDLER_LIST(DATA_HANDLER_MAP_ACCESSOR)
-#undef DATA_HANDLER_MAP_ACCESSOR
-
-#define STRING_ACCESSOR(name, str) \
- Handle<String> Factory::name() { \
- return Handle<String>(bit_cast<String**>( \
- &isolate()->heap()->roots_[Heap::k##name##RootIndex])); \
- }
-INTERNALIZED_STRING_LIST(STRING_ACCESSOR)
-#undef STRING_ACCESSOR
-
-#define SYMBOL_ACCESSOR(name) \
- Handle<Symbol> Factory::name() { \
- return Handle<Symbol>(bit_cast<Symbol**>( \
- &isolate()->heap()->roots_[Heap::k##name##RootIndex])); \
- }
-PRIVATE_SYMBOL_LIST(SYMBOL_ACCESSOR)
-#undef SYMBOL_ACCESSOR
-
-#define SYMBOL_ACCESSOR(name, description) \
- Handle<Symbol> Factory::name() { \
- return Handle<Symbol>(bit_cast<Symbol**>( \
- &isolate()->heap()->roots_[Heap::k##name##RootIndex])); \
- }
-PUBLIC_SYMBOL_LIST(SYMBOL_ACCESSOR)
-WELL_KNOWN_SYMBOL_LIST(SYMBOL_ACCESSOR)
-#undef SYMBOL_ACCESSOR
-
-#define ACCESSOR_INFO_ACCESSOR(accessor_name, AccessorName) \
- Handle<AccessorInfo> Factory::accessor_name##_accessor() { \
- return Handle<AccessorInfo>(bit_cast<AccessorInfo**>( \
- &isolate() \
- ->heap() \
- ->roots_[Heap::k##AccessorName##AccessorRootIndex])); \
- }
-ACCESSOR_INFO_LIST(ACCESSOR_INFO_ACCESSOR)
-#undef ACCESSOR_INFO_ACCESSOR
-
Handle<String> Factory::InternalizeString(Handle<String> string) {
if (string->IsInternalizedString()) return string;
return StringTable::LookupString(isolate(), string);
diff --git a/deps/v8/src/heap/factory.cc b/deps/v8/src/heap/factory.cc
index c8528f9fdb..9535eb4b88 100644
--- a/deps/v8/src/heap/factory.cc
+++ b/deps/v8/src/heap/factory.cc
@@ -27,9 +27,11 @@
#include "src/objects/js-regexp-inl.h"
#include "src/objects/literal-objects-inl.h"
#include "src/objects/microtask-inl.h"
+#include "src/objects/microtask-queue-inl.h"
#include "src/objects/module-inl.h"
#include "src/objects/promise-inl.h"
#include "src/objects/scope-info.h"
+#include "src/objects/stack-frame-info-inl.h"
#include "src/unicode-cache.h"
#include "src/unicode-decoder.h"
@@ -63,9 +65,9 @@ void InitializeCode(Heap* heap, Handle<Code> code, int object_size,
bool is_turbofanned, int stack_slots,
int safepoint_table_offset, int handler_table_offset) {
DCHECK(IsAligned(code->address(), kCodeAlignment));
- DCHECK(!heap->memory_allocator()->code_range()->valid() ||
- heap->memory_allocator()->code_range()->contains(code->address()) ||
- object_size <= heap->code_space()->AreaSize());
+ DCHECK_IMPLIES(
+ !heap->memory_allocator()->code_range().is_empty(),
+ heap->memory_allocator()->code_range().contains(code->address()));
bool has_unwinding_info = desc.unwinding_info != nullptr;
@@ -287,9 +289,9 @@ Handle<PropertyArray> Factory::NewPropertyArray(int length,
return array;
}
-Handle<FixedArray> Factory::NewFixedArrayWithFiller(
- Heap::RootListIndex map_root_index, int length, Object* filler,
- PretenureFlag pretenure) {
+Handle<FixedArray> Factory::NewFixedArrayWithFiller(RootIndex map_root_index,
+ int length, Object* filler,
+ PretenureFlag pretenure) {
HeapObject* result = AllocateRawFixedArray(length, pretenure);
DCHECK(Heap::RootIsImmortalImmovable(map_root_index));
Map* map = Map::cast(isolate()->heap()->root(map_root_index));
@@ -301,8 +303,8 @@ Handle<FixedArray> Factory::NewFixedArrayWithFiller(
}
template <typename T>
-Handle<T> Factory::NewFixedArrayWithMap(Heap::RootListIndex map_root_index,
- int length, PretenureFlag pretenure) {
+Handle<T> Factory::NewFixedArrayWithMap(RootIndex map_root_index, int length,
+ PretenureFlag pretenure) {
static_assert(std::is_base_of<FixedArray, T>::value,
"T must be a descendant of FixedArray");
// Zero-length case must be handled outside, where the knowledge about
@@ -313,7 +315,7 @@ Handle<T> Factory::NewFixedArrayWithMap(Heap::RootListIndex map_root_index,
}
template <typename T>
-Handle<T> Factory::NewWeakFixedArrayWithMap(Heap::RootListIndex map_root_index,
+Handle<T> Factory::NewWeakFixedArrayWithMap(RootIndex map_root_index,
int length,
PretenureFlag pretenure) {
static_assert(std::is_base_of<WeakFixedArray, T>::value,
@@ -336,16 +338,16 @@ Handle<T> Factory::NewWeakFixedArrayWithMap(Heap::RootListIndex map_root_index,
}
template Handle<FixedArray> Factory::NewFixedArrayWithMap<FixedArray>(
- Heap::RootListIndex, int, PretenureFlag);
+ RootIndex, int, PretenureFlag);
template Handle<DescriptorArray>
-Factory::NewWeakFixedArrayWithMap<DescriptorArray>(Heap::RootListIndex, int,
+Factory::NewWeakFixedArrayWithMap<DescriptorArray>(RootIndex, int,
PretenureFlag);
Handle<FixedArray> Factory::NewFixedArray(int length, PretenureFlag pretenure) {
DCHECK_LE(0, length);
if (length == 0) return empty_fixed_array();
- return NewFixedArrayWithFiller(Heap::kFixedArrayMapRootIndex, length,
+ return NewFixedArrayWithFiller(RootIndex::kFixedArrayMap, length,
*undefined_value(), pretenure);
}
@@ -355,7 +357,7 @@ Handle<WeakFixedArray> Factory::NewWeakFixedArray(int length,
if (length == 0) return empty_weak_fixed_array();
HeapObject* result =
AllocateRawArray(WeakFixedArray::SizeFor(length), pretenure);
- DCHECK(Heap::RootIsImmortalImmovable(Heap::kWeakFixedArrayMapRootIndex));
+ DCHECK(Heap::RootIsImmortalImmovable(RootIndex::kWeakFixedArrayMap));
result->set_map_after_allocation(*weak_fixed_array_map(), SKIP_WRITE_BARRIER);
Handle<WeakFixedArray> array(WeakFixedArray::cast(result), isolate());
array->set_length(length);
@@ -391,7 +393,7 @@ Handle<FixedArray> Factory::NewFixedArrayWithHoles(int length,
PretenureFlag pretenure) {
DCHECK_LE(0, length);
if (length == 0) return empty_fixed_array();
- return NewFixedArrayWithFiller(Heap::kFixedArrayMapRootIndex, length,
+ return NewFixedArrayWithFiller(RootIndex::kFixedArrayMap, length,
*the_hole_value(), pretenure);
}
@@ -403,7 +405,7 @@ Handle<FixedArray> Factory::NewUninitializedFixedArray(
// TODO(ulan): As an experiment this temporarily returns an initialized fixed
// array. After getting canary/performance coverage, either remove the
// function or revert to returning uninitilized array.
- return NewFixedArrayWithFiller(Heap::kFixedArrayMapRootIndex, length,
+ return NewFixedArrayWithFiller(RootIndex::kFixedArrayMap, length,
*undefined_value(), pretenure);
}
@@ -452,7 +454,7 @@ Handle<ObjectBoilerplateDescription> Factory::NewObjectBoilerplateDescription(
Handle<ObjectBoilerplateDescription> description =
Handle<ObjectBoilerplateDescription>::cast(NewFixedArrayWithMap(
- Heap::kObjectBoilerplateDescriptionMapRootIndex, size, TENURED));
+ RootIndex::kObjectBoilerplateDescriptionMap, size, TENURED));
if (has_different_size_backing_store) {
DCHECK_IMPLIES((boilerplate == (all_properties - index_keys)),
@@ -773,7 +775,7 @@ Handle<SeqOneByteString> Factory::AllocateRawOneByteInternalizedString(
// The canonical empty_string is the only zero-length string we allow.
DCHECK_IMPLIES(
length == 0,
- isolate()->heap()->roots_[Heap::kempty_stringRootIndex] == nullptr);
+ isolate()->heap()->roots_[RootIndex::kempty_string] == nullptr);
Map* map = *one_byte_internalized_string_map();
int size = SeqOneByteString::SizeFor(length);
@@ -900,12 +902,12 @@ MaybeHandle<Map> GetInternalizedStringMap(Factory* f, Handle<String> string) {
return f->external_one_byte_internalized_string_map();
case EXTERNAL_STRING_WITH_ONE_BYTE_DATA_TYPE:
return f->external_internalized_string_with_one_byte_data_map();
- case SHORT_EXTERNAL_STRING_TYPE:
- return f->short_external_internalized_string_map();
- case SHORT_EXTERNAL_ONE_BYTE_STRING_TYPE:
- return f->short_external_one_byte_internalized_string_map();
- case SHORT_EXTERNAL_STRING_WITH_ONE_BYTE_DATA_TYPE:
- return f->short_external_internalized_string_with_one_byte_data_map();
+ case UNCACHED_EXTERNAL_STRING_TYPE:
+ return f->uncached_external_internalized_string_map();
+ case UNCACHED_EXTERNAL_ONE_BYTE_STRING_TYPE:
+ return f->uncached_external_one_byte_internalized_string_map();
+ case UNCACHED_EXTERNAL_STRING_WITH_ONE_BYTE_DATA_TYPE:
+ return f->uncached_external_internalized_string_with_one_byte_data_map();
default:
return MaybeHandle<Map>(); // No match found.
}
@@ -1083,7 +1085,7 @@ MaybeHandle<String> Factory::NewConsString(Handle<String> left,
bool is_one_byte_data_in_two_byte_string = false;
if (!is_one_byte) {
// At least one of the strings uses two-byte representation so we
- // can't use the fast case code for short one-byte strings below, but
+ // can't use the fast case code for uncached one-byte strings below, but
// we can try to save memory if all chars actually fit in one-byte.
is_one_byte_data_in_two_byte_string =
left->HasOnlyOneByteChars() && right->HasOnlyOneByteChars();
@@ -1243,9 +1245,8 @@ MaybeHandle<String> Factory::NewExternalStringFromOneByte(
if (length == 0) return empty_string();
Handle<Map> map;
- if (resource->IsCompressible()) {
- // TODO(hajimehoshi): Rename this to 'uncached_external_one_byte_string_map'
- map = short_external_one_byte_string_map();
+ if (!resource->IsCacheable()) {
+ map = uncached_external_one_byte_string_map();
} else {
map = external_one_byte_string_map();
}
@@ -1274,10 +1275,9 @@ MaybeHandle<String> Factory::NewExternalStringFromTwoByte(
length <= kOneByteCheckLengthLimit &&
String::IsOneByte(resource->data(), static_cast<int>(length));
Handle<Map> map;
- if (resource->IsCompressible()) {
- // TODO(hajimehoshi): Rename these to 'uncached_external_string_...'.
- map = is_one_byte ? short_external_string_with_one_byte_data_map()
- : short_external_string_map();
+ if (!resource->IsCacheable()) {
+ map = is_one_byte ? uncached_external_string_with_one_byte_data_map()
+ : uncached_external_string_map();
} else {
map = is_one_byte ? external_string_with_one_byte_data_map()
: external_string_map();
@@ -1309,7 +1309,7 @@ Handle<ExternalOneByteString> Factory::NewNativeSourceString(
}
Handle<JSStringIterator> Factory::NewJSStringIterator(Handle<String> string) {
- Handle<Map> map(isolate()->native_context()->string_iterator_map(),
+ Handle<Map> map(isolate()->native_context()->initial_string_iterator_map(),
isolate());
Handle<String> flat_string = String::Flatten(isolate(), string);
Handle<JSStringIterator> iterator =
@@ -1355,7 +1355,7 @@ Handle<Symbol> Factory::NewPrivateFieldSymbol() {
Handle<NativeContext> Factory::NewNativeContext() {
Handle<NativeContext> context = NewFixedArrayWithMap<NativeContext>(
- Heap::kNativeContextMapRootIndex, Context::NATIVE_CONTEXT_SLOTS, TENURED);
+ RootIndex::kNativeContextMap, Context::NATIVE_CONTEXT_SLOTS, TENURED);
context->set_native_context(*context);
context->set_errors_thrown(Smi::kZero);
context->set_math_random_index(Smi::kZero);
@@ -1367,7 +1367,7 @@ Handle<Context> Factory::NewScriptContext(Handle<NativeContext> outer,
Handle<ScopeInfo> scope_info) {
DCHECK_EQ(scope_info->scope_type(), SCRIPT_SCOPE);
Handle<Context> context = NewFixedArrayWithMap<Context>(
- Heap::kScriptContextMapRootIndex, scope_info->ContextLength(), TENURED);
+ RootIndex::kScriptContextMap, scope_info->ContextLength(), TENURED);
context->set_scope_info(*scope_info);
context->set_previous(*outer);
context->set_extension(*the_hole_value());
@@ -1379,8 +1379,7 @@ Handle<Context> Factory::NewScriptContext(Handle<NativeContext> outer,
Handle<ScriptContextTable> Factory::NewScriptContextTable() {
Handle<ScriptContextTable> context_table =
NewFixedArrayWithMap<ScriptContextTable>(
- Heap::kScriptContextTableMapRootIndex,
- ScriptContextTable::kMinLength);
+ RootIndex::kScriptContextTableMap, ScriptContextTable::kMinLength);
context_table->set_used(0);
return context_table;
}
@@ -1390,7 +1389,7 @@ Handle<Context> Factory::NewModuleContext(Handle<Module> module,
Handle<ScopeInfo> scope_info) {
DCHECK_EQ(scope_info->scope_type(), MODULE_SCOPE);
Handle<Context> context = NewFixedArrayWithMap<Context>(
- Heap::kModuleContextMapRootIndex, scope_info->ContextLength(), TENURED);
+ RootIndex::kModuleContextMap, scope_info->ContextLength(), TENURED);
context->set_scope_info(*scope_info);
context->set_previous(*outer);
context->set_extension(*module);
@@ -1403,13 +1402,13 @@ Handle<Context> Factory::NewFunctionContext(Handle<Context> outer,
Handle<ScopeInfo> scope_info) {
int length = scope_info->ContextLength();
DCHECK_LE(Context::MIN_CONTEXT_SLOTS, length);
- Heap::RootListIndex mapRootIndex;
+ RootIndex mapRootIndex;
switch (scope_info->scope_type()) {
case EVAL_SCOPE:
- mapRootIndex = Heap::kEvalContextMapRootIndex;
+ mapRootIndex = RootIndex::kEvalContextMap;
break;
case FUNCTION_SCOPE:
- mapRootIndex = Heap::kFunctionContextMapRootIndex;
+ mapRootIndex = RootIndex::kFunctionContextMap;
break;
default:
UNREACHABLE();
@@ -1427,7 +1426,7 @@ Handle<Context> Factory::NewCatchContext(Handle<Context> previous,
Handle<Object> thrown_object) {
STATIC_ASSERT(Context::MIN_CONTEXT_SLOTS == Context::THROWN_OBJECT_INDEX);
Handle<Context> context = NewFixedArrayWithMap<Context>(
- Heap::kCatchContextMapRootIndex, Context::MIN_CONTEXT_SLOTS + 1);
+ RootIndex::kCatchContextMap, Context::MIN_CONTEXT_SLOTS + 1);
context->set_scope_info(*scope_info);
context->set_previous(*previous);
context->set_extension(*the_hole_value());
@@ -1447,7 +1446,7 @@ Handle<Context> Factory::NewDebugEvaluateContext(Handle<Context> previous,
? Handle<HeapObject>::cast(the_hole_value())
: Handle<HeapObject>::cast(extension);
Handle<Context> c = NewFixedArrayWithMap<Context>(
- Heap::kDebugEvaluateContextMapRootIndex, Context::MIN_CONTEXT_SLOTS + 2);
+ RootIndex::kDebugEvaluateContextMap, Context::MIN_CONTEXT_SLOTS + 2);
c->set_scope_info(*scope_info);
c->set_previous(*previous);
c->set_native_context(previous->native_context());
@@ -1461,7 +1460,7 @@ Handle<Context> Factory::NewWithContext(Handle<Context> previous,
Handle<ScopeInfo> scope_info,
Handle<JSReceiver> extension) {
Handle<Context> context = NewFixedArrayWithMap<Context>(
- Heap::kWithContextMapRootIndex, Context::MIN_CONTEXT_SLOTS);
+ RootIndex::kWithContextMap, Context::MIN_CONTEXT_SLOTS);
context->set_scope_info(*scope_info);
context->set_previous(*previous);
context->set_extension(*extension);
@@ -1473,7 +1472,7 @@ Handle<Context> Factory::NewBlockContext(Handle<Context> previous,
Handle<ScopeInfo> scope_info) {
DCHECK_EQ(scope_info->scope_type(), BLOCK_SCOPE);
Handle<Context> context = NewFixedArrayWithMap<Context>(
- Heap::kBlockContextMapRootIndex, scope_info->ContextLength());
+ RootIndex::kBlockContextMap, scope_info->ContextLength());
context->set_scope_info(*scope_info);
context->set_previous(*previous);
context->set_extension(*the_hole_value());
@@ -1485,7 +1484,7 @@ Handle<Context> Factory::NewBuiltinContext(Handle<NativeContext> native_context,
int length) {
DCHECK_GE(length, Context::MIN_CONTEXT_SLOTS);
Handle<Context> context =
- NewFixedArrayWithMap<Context>(Heap::kFunctionContextMapRootIndex, length);
+ NewFixedArrayWithMap<Context>(RootIndex::kFunctionContextMap, length);
context->set_scope_info(ReadOnlyRoots(isolate()).empty_scope_info());
context->set_extension(*the_hole_value());
context->set_native_context(*native_context);
@@ -1495,8 +1494,8 @@ Handle<Context> Factory::NewBuiltinContext(Handle<NativeContext> native_context,
Handle<Struct> Factory::NewStruct(InstanceType type, PretenureFlag pretenure) {
Map* map;
switch (type) {
-#define MAKE_CASE(NAME, Name, name) \
- case NAME##_TYPE: \
+#define MAKE_CASE(TYPE, Name, name) \
+ case TYPE: \
map = *name##_map(); \
break;
STRUCT_LIST(MAKE_CASE)
@@ -1623,6 +1622,16 @@ Handle<PromiseResolveThenableJobTask> Factory::NewPromiseResolveThenableJobTask(
return microtask;
}
+Handle<MicrotaskQueue> Factory::NewMicrotaskQueue() {
+ // MicrotaskQueue should be TENURED, as it outlives Context, and is mostly
+ // as long-living as Context is.
+ Handle<MicrotaskQueue> microtask_queue =
+ Handle<MicrotaskQueue>::cast(NewStruct(MICROTASK_QUEUE_TYPE, TENURED));
+ microtask_queue->set_queue(*empty_fixed_array());
+ microtask_queue->set_pending_microtask_count(0);
+ return microtask_queue;
+}
+
Handle<Foreign> Factory::NewForeign(Address addr, PretenureFlag pretenure) {
// Statically ensure that it is safe to allocate foreigns in paged spaces.
STATIC_ASSERT(Foreign::kSize <= kMaxRegularHeapObjectSize);
@@ -1687,7 +1696,8 @@ Handle<FixedTypedArrayBase> Factory::NewFixedTypedArrayWithExternalPointer(
DCHECK(0 <= length && length <= Smi::kMaxValue);
int size = FixedTypedArrayBase::kHeaderSize;
HeapObject* result = AllocateRawWithImmortalMap(
- size, pretenure, isolate()->heap()->MapForFixedTypedArray(array_type));
+ size, pretenure,
+ ReadOnlyRoots(isolate()).MapForFixedTypedArray(array_type));
Handle<FixedTypedArrayBase> elements(FixedTypedArrayBase::cast(result),
isolate());
elements->set_base_pointer(Smi::kZero, SKIP_WRITE_BARRIER);
@@ -1704,7 +1714,7 @@ Handle<FixedTypedArrayBase> Factory::NewFixedTypedArray(
CHECK(byte_length <= kMaxInt - FixedTypedArrayBase::kDataOffset);
size_t size =
OBJECT_POINTER_ALIGN(byte_length + FixedTypedArrayBase::kDataOffset);
- Map* map = isolate()->heap()->MapForFixedTypedArray(array_type);
+ Map* map = ReadOnlyRoots(isolate()).MapForFixedTypedArray(array_type);
AllocationAlignment alignment =
array_type == kExternalFloat64Array ? kDoubleAligned : kWordAligned;
HeapObject* object = AllocateRawWithImmortalMap(static_cast<int>(size),
@@ -1778,7 +1788,7 @@ Handle<TransitionArray> Factory::NewTransitionArray(int number_of_transitions,
int slack) {
int capacity = TransitionArray::LengthFor(number_of_transitions + slack);
Handle<TransitionArray> array = NewWeakFixedArrayWithMap<TransitionArray>(
- Heap::kTransitionArrayMapRootIndex, capacity, TENURED);
+ RootIndex::kTransitionArrayMap, capacity, TENURED);
// Transition arrays are tenured. When black allocation is on we have to
// add the transition array to the list of encountered_transition_arrays.
Heap* heap = isolate()->heap();
@@ -1812,7 +1822,7 @@ Handle<Map> Factory::NewMap(InstanceType type, int instance_size,
ElementsKind elements_kind,
int inobject_properties) {
STATIC_ASSERT(LAST_JS_OBJECT_TYPE == LAST_TYPE);
- DCHECK_IMPLIES(Map::IsJSObject(type) &&
+ DCHECK_IMPLIES(InstanceTypeChecker::IsJSObject(type) &&
!Map::CanHaveFastTransitionableElementsKind(type),
IsDictionaryElementsKind(elements_kind) ||
IsTerminalElementsKind(elements_kind));
@@ -2480,12 +2490,12 @@ Handle<JSFunction> Factory::NewFunctionFromSharedFunctionInfo(
}
Handle<ScopeInfo> Factory::NewScopeInfo(int length) {
- return NewFixedArrayWithMap<ScopeInfo>(Heap::kScopeInfoMapRootIndex, length,
+ return NewFixedArrayWithMap<ScopeInfo>(RootIndex::kScopeInfoMap, length,
TENURED);
}
Handle<ModuleInfo> Factory::NewModuleInfo() {
- return NewFixedArrayWithMap<ModuleInfo>(Heap::kModuleInfoMapRootIndex,
+ return NewFixedArrayWithMap<ModuleInfo>(RootIndex::kModuleInfoMap,
ModuleInfo::kLength, TENURED);
}
@@ -2665,9 +2675,9 @@ Handle<Code> Factory::NewCodeForDeserialization(uint32_t size) {
heap->ZapCodeObject(result->address(), size);
result->set_map_after_allocation(*code_map(), SKIP_WRITE_BARRIER);
DCHECK(IsAligned(result->address(), kCodeAlignment));
- DCHECK(!heap->memory_allocator()->code_range()->valid() ||
- heap->memory_allocator()->code_range()->contains(result->address()) ||
- static_cast<int>(size) <= heap->code_space()->AreaSize());
+ DCHECK_IMPLIES(
+ !heap->memory_allocator()->code_range().is_empty(),
+ heap->memory_allocator()->code_range().contains(result->address()));
return handle(Code::cast(result), isolate());
}
@@ -2729,10 +2739,9 @@ Handle<Code> Factory::CopyCode(Handle<Code> code) {
if (FLAG_verify_heap) new_code->ObjectVerify(isolate());
#endif
DCHECK(IsAligned(new_code->address(), kCodeAlignment));
- DCHECK(
- !heap->memory_allocator()->code_range()->valid() ||
- heap->memory_allocator()->code_range()->contains(new_code->address()) ||
- obj_size <= heap->code_space()->AreaSize());
+ DCHECK_IMPLIES(
+ !heap->memory_allocator()->code_range().is_empty(),
+ heap->memory_allocator()->code_range().contains(new_code->address()));
return new_code;
}
@@ -3095,26 +3104,6 @@ Handle<JSSet> Factory::NewJSSet() {
return js_set;
}
-Handle<JSMapIterator> Factory::NewJSMapIterator(Handle<Map> map,
- Handle<OrderedHashMap> table,
- int index) {
- Handle<JSMapIterator> result =
- Handle<JSMapIterator>::cast(NewJSObjectFromMap(map));
- result->set_table(*table);
- result->set_index(Smi::FromInt(index));
- return result;
-}
-
-Handle<JSSetIterator> Factory::NewJSSetIterator(Handle<Map> map,
- Handle<OrderedHashSet> table,
- int index) {
- Handle<JSSetIterator> result =
- Handle<JSSetIterator>::cast(NewJSObjectFromMap(map));
- result->set_table(*table);
- result->set_index(Smi::FromInt(index));
- return result;
-}
-
void Factory::TypeAndSizeForElementsKind(ElementsKind kind,
ExternalArrayType* array_type,
size_t* element_size) {
@@ -3181,26 +3170,16 @@ JSFunction* GetTypedArrayFun(ElementsKind elements_kind, Isolate* isolate) {
void SetupArrayBufferView(i::Isolate* isolate,
i::Handle<i::JSArrayBufferView> obj,
i::Handle<i::JSArrayBuffer> buffer,
- size_t byte_offset, size_t byte_length,
- PretenureFlag pretenure = NOT_TENURED) {
- DCHECK(byte_offset + byte_length <=
- static_cast<size_t>(buffer->byte_length()->Number()));
-
+ size_t byte_offset, size_t byte_length) {
+ DCHECK_LE(byte_offset + byte_length, buffer->byte_length());
DCHECK_EQ(obj->GetEmbedderFieldCount(),
v8::ArrayBufferView::kEmbedderFieldCount);
for (int i = 0; i < v8::ArrayBufferView::kEmbedderFieldCount; i++) {
obj->SetEmbedderField(i, Smi::kZero);
}
-
obj->set_buffer(*buffer);
-
- i::Handle<i::Object> byte_offset_object =
- isolate->factory()->NewNumberFromSize(byte_offset, pretenure);
- obj->set_byte_offset(*byte_offset_object);
-
- i::Handle<i::Object> byte_length_object =
- isolate->factory()->NewNumberFromSize(byte_length, pretenure);
- obj->set_byte_length(*byte_length_object);
+ obj->set_byte_offset(byte_offset);
+ obj->set_byte_length(byte_length);
}
} // namespace
@@ -3237,8 +3216,7 @@ Handle<JSTypedArray> Factory::NewJSTypedArray(ExternalArrayType type,
// TODO(7881): Smi length check
CHECK(length <= static_cast<size_t>(Smi::kMaxValue));
size_t byte_length = length * element_size;
- SetupArrayBufferView(isolate(), obj, buffer, byte_offset, byte_length,
- pretenure);
+ SetupArrayBufferView(isolate(), obj, buffer, byte_offset, byte_length);
Handle<Object> length_object = NewNumberFromSize(length, pretenure);
obj->set_length(*length_object);
@@ -3271,13 +3249,9 @@ Handle<JSTypedArray> Factory::NewJSTypedArray(ElementsKind elements_kind,
CHECK(number_of_elements <= static_cast<size_t>(Smi::kMaxValue));
size_t byte_length = number_of_elements * element_size;
- obj->set_byte_offset(Smi::kZero);
- i::Handle<i::Object> byte_length_object =
- NewNumberFromSize(byte_length, pretenure);
- obj->set_byte_length(*byte_length_object);
- Handle<Object> length_object =
- NewNumberFromSize(number_of_elements, pretenure);
- obj->set_length(*length_object);
+ obj->set_byte_offset(0);
+ obj->set_byte_length(byte_length);
+ obj->set_length(Smi::FromIntptr(static_cast<intptr_t>(number_of_elements)));
Handle<JSArrayBuffer> buffer =
NewJSArrayBuffer(SharedFlag::kNotShared, pretenure);
@@ -3757,7 +3731,7 @@ Handle<Map> Factory::ObjectLiteralMapFromCache(Handle<NativeContext> context,
Handle<WeakFixedArray> cache = Handle<WeakFixedArray>::cast(maybe_cache);
MaybeObject* result = cache->Get(cache_index);
HeapObject* heap_object;
- if (result->ToWeakHeapObject(&heap_object)) {
+ if (result->GetHeapObjectIfWeak(&heap_object)) {
Map* map = Map::cast(heap_object);
DCHECK(!map->is_dictionary_map());
return handle(map, isolate());
diff --git a/deps/v8/src/heap/factory.h b/deps/v8/src/heap/factory.h
index cd57b5bf87..8c6d32090e 100644
--- a/deps/v8/src/heap/factory.h
+++ b/deps/v8/src/heap/factory.h
@@ -44,6 +44,7 @@ class JSGeneratorObject;
class JSMap;
class JSMapIterator;
class JSModuleNamespace;
+class JSPromise;
class JSProxy;
class JSSet;
class JSSetIterator;
@@ -56,6 +57,7 @@ class PreParsedScopeData;
class PromiseResolveThenableJobTask;
class RegExpMatchInfo;
class ScriptContextTable;
+class StackFrameInfo;
class StoreHandler;
class TemplateObjectDescription;
class UncompiledDataWithoutPreParsedScope;
@@ -107,14 +109,13 @@ class V8_EXPORT_PRIVATE Factory {
// Allocates a fixed array-like object with given map and initialized with
// undefined values.
template <typename T = FixedArray>
- Handle<T> NewFixedArrayWithMap(Heap::RootListIndex map_root_index, int length,
+ Handle<T> NewFixedArrayWithMap(RootIndex map_root_index, int length,
PretenureFlag pretenure = NOT_TENURED);
// Allocates a weak fixed array-like object with given map and initialized
// with undefined values.
template <typename T = WeakFixedArray>
- Handle<T> NewWeakFixedArrayWithMap(Heap::RootListIndex map_root_index,
- int length,
+ Handle<T> NewWeakFixedArrayWithMap(RootIndex map_root_index, int length,
PretenureFlag pretenure = NOT_TENURED);
// Allocates a fixed array initialized with undefined values.
@@ -439,6 +440,8 @@ class V8_EXPORT_PRIVATE Factory {
Handle<JSPromise> promise_to_resolve, Handle<JSReceiver> then,
Handle<JSReceiver> thenable, Handle<Context> context);
+ Handle<MicrotaskQueue> NewMicrotaskQueue();
+
// Foreign objects are pretenured when allocated by the bootstrapper.
Handle<Foreign> NewForeign(Address addr,
PretenureFlag pretenure = NOT_TENURED);
@@ -667,13 +670,6 @@ class V8_EXPORT_PRIVATE Factory {
Handle<JSMap> NewJSMap();
Handle<JSSet> NewJSSet();
- Handle<JSMapIterator> NewJSMapIterator(Handle<Map> map,
- Handle<OrderedHashMap> table,
- int index);
- Handle<JSSetIterator> NewJSSetIterator(Handle<Map> map,
- Handle<OrderedHashSet> table,
- int index);
-
// Allocates a bound function.
MaybeHandle<JSBoundFunction> NewJSBoundFunction(
Handle<JSReceiver> target_function, Handle<Object> bound_this,
@@ -828,45 +824,12 @@ class V8_EXPORT_PRIVATE Factory {
Handle<String> NumberToString(Handle<Object> number, bool check_cache = true);
Handle<String> NumberToString(Smi* number, bool check_cache = true);
- inline Handle<String> Uint32ToString(uint32_t value,
- bool check_cache = false);
+ inline Handle<String> Uint32ToString(uint32_t value, bool check_cache = true);
-#define ROOT_ACCESSOR(type, name, camel_name) inline Handle<type> name();
+#define ROOT_ACCESSOR(type, name, CamelName) inline Handle<type> name();
ROOT_LIST(ROOT_ACCESSOR)
#undef ROOT_ACCESSOR
-#define STRUCT_MAP_ACCESSOR(NAME, Name, name) inline Handle<Map> name##_map();
- STRUCT_LIST(STRUCT_MAP_ACCESSOR)
-#undef STRUCT_MAP_ACCESSOR
-
-#define ALLOCATION_SITE_MAP_ACCESSOR(NAME, Name, Size, name) \
- inline Handle<Map> name##_map();
- ALLOCATION_SITE_LIST(ALLOCATION_SITE_MAP_ACCESSOR)
-#undef ALLOCATION_SITE_MAP_ACCESSOR
-
-#define DATA_HANDLER_MAP_ACCESSOR(NAME, Name, Size, name) \
- inline Handle<Map> name##_map();
- DATA_HANDLER_LIST(DATA_HANDLER_MAP_ACCESSOR)
-#undef DATA_HANDLER_MAP_ACCESSOR
-
-#define STRING_ACCESSOR(name, str) inline Handle<String> name();
- INTERNALIZED_STRING_LIST(STRING_ACCESSOR)
-#undef STRING_ACCESSOR
-
-#define SYMBOL_ACCESSOR(name) inline Handle<Symbol> name();
- PRIVATE_SYMBOL_LIST(SYMBOL_ACCESSOR)
-#undef SYMBOL_ACCESSOR
-
-#define SYMBOL_ACCESSOR(name, description) inline Handle<Symbol> name();
- PUBLIC_SYMBOL_LIST(SYMBOL_ACCESSOR)
- WELL_KNOWN_SYMBOL_LIST(SYMBOL_ACCESSOR)
-#undef SYMBOL_ACCESSOR
-
-#define ACCESSOR_INFO_ACCESSOR(accessor_name, AccessorName) \
- inline Handle<AccessorInfo> accessor_name##_accessor();
- ACCESSOR_INFO_LIST(ACCESSOR_INFO_ACCESSOR)
-#undef ACCESSOR_INFO_ACCESSOR
-
// Allocates a new SharedFunctionInfo object.
Handle<SharedFunctionInfo> NewSharedFunctionInfoForApiFunction(
MaybeHandle<String> maybe_name,
@@ -961,6 +924,7 @@ class V8_EXPORT_PRIVATE Factory {
// Downcast to the privately inherited sub-class using c-style casts to
// avoid undefined behavior (as static_cast cannot cast across private
// bases).
+ // NOLINTNEXTLINE (google-readability-casting)
return (Isolate*)this; // NOLINT(readability/casting)
}
@@ -975,7 +939,7 @@ class V8_EXPORT_PRIVATE Factory {
HeapObject* AllocateRawArray(int size, PretenureFlag pretenure);
HeapObject* AllocateRawFixedArray(int length, PretenureFlag pretenure);
HeapObject* AllocateRawWeakArrayList(int length, PretenureFlag pretenure);
- Handle<FixedArray> NewFixedArrayWithFiller(Heap::RootListIndex map_root_index,
+ Handle<FixedArray> NewFixedArrayWithFiller(RootIndex map_root_index,
int length, Object* filler,
PretenureFlag pretenure);
@@ -1054,7 +1018,7 @@ class NewFunctionArgs final {
Handle<Map> GetMap(Isolate* isolate) const;
private:
- NewFunctionArgs() {} // Use the static factory constructors.
+ NewFunctionArgs() = default; // Use the static factory constructors.
void SetShouldCreateAndSetInitialMap();
void SetShouldSetPrototype();
diff --git a/deps/v8/src/heap/gc-tracer.cc b/deps/v8/src/heap/gc-tracer.cc
index 5ee7186c6a..7d33c68ad1 100644
--- a/deps/v8/src/heap/gc-tracer.cc
+++ b/deps/v8/src/heap/gc-tracer.cc
@@ -1093,7 +1093,7 @@ void GCTracer::AddBackgroundScopeSample(
}
}
-void GCTracer::RecordMarkCompactHistograms(HistogramTimer* gc_timer) {
+void GCTracer::RecordGCPhasesHistograms(HistogramTimer* gc_timer) {
Counters* counters = heap_->isolate()->counters();
if (gc_timer == counters->gc_finalize()) {
DCHECK_EQ(Scope::FIRST_TOP_MC_SCOPE, Scope::MC_CLEAR);
@@ -1112,6 +1112,11 @@ void GCTracer::RecordMarkCompactHistograms(HistogramTimer* gc_timer) {
counters->gc_finalize_sweep()->AddSample(
static_cast<int>(current_.scopes[Scope::MC_SWEEP]));
DCHECK_EQ(Scope::LAST_TOP_MC_SCOPE, Scope::MC_SWEEP);
+ } else if (gc_timer == counters->gc_scavenger()) {
+ counters->gc_scavenger_scavenge_main()->AddSample(
+ static_cast<int>(current_.scopes[Scope::SCAVENGER_SCAVENGE_PARALLEL]));
+ counters->gc_scavenger_scavenge_roots()->AddSample(
+ static_cast<int>(current_.scopes[Scope::SCAVENGER_SCAVENGE_ROOTS]));
}
}
diff --git a/deps/v8/src/heap/gc-tracer.h b/deps/v8/src/heap/gc-tracer.h
index 62e077be50..bf49586d57 100644
--- a/deps/v8/src/heap/gc-tracer.h
+++ b/deps/v8/src/heap/gc-tracer.h
@@ -321,7 +321,7 @@ class V8_EXPORT_PRIVATE GCTracer {
void AddBackgroundScopeSample(BackgroundScope::ScopeId scope, double duration,
RuntimeCallCounter* runtime_call_counter);
- void RecordMarkCompactHistograms(HistogramTimer* gc_timer);
+ void RecordGCPhasesHistograms(HistogramTimer* gc_timer);
private:
FRIEND_TEST(GCTracer, AverageSpeed);
@@ -339,6 +339,7 @@ class V8_EXPORT_PRIVATE GCTracer {
FRIEND_TEST(GCTracerTest, IncrementalMarkingSpeed);
FRIEND_TEST(GCTracerTest, MutatorUtilization);
FRIEND_TEST(GCTracerTest, RecordMarkCompactHistograms);
+ FRIEND_TEST(GCTracerTest, RecordScavengerHistograms);
struct BackgroundCounter {
double total_duration_ms;
diff --git a/deps/v8/src/heap/heap-controller.cc b/deps/v8/src/heap/heap-controller.cc
index 485b22902a..d515199518 100644
--- a/deps/v8/src/heap/heap-controller.cc
+++ b/deps/v8/src/heap/heap-controller.cc
@@ -10,16 +10,16 @@ namespace internal {
// Given GC speed in bytes per ms, the allocation throughput in bytes per ms
// (mutator speed), this function returns the heap growing factor that will
-// achieve the kTargetMutatorUtilisation if the GC speed and the mutator speed
+// achieve the target_mutator_utilization_ if the GC speed and the mutator speed
// remain the same until the next GC.
//
// For a fixed time-frame T = TM + TG, the mutator utilization is the ratio
// TM / (TM + TG), where TM is the time spent in the mutator and TG is the
// time spent in the garbage collector.
//
-// Let MU be kTargetMutatorUtilisation, the desired mutator utilization for the
-// time-frame from the end of the current GC to the end of the next GC. Based
-// on the MU we can compute the heap growing factor F as
+// Let MU be target_mutator_utilization_, the desired mutator utilization for
+// the time-frame from the end of the current GC to the end of the next GC.
+// Based on the MU we can compute the heap growing factor F as
//
// F = R * (1 - MU) / (R * (1 - MU) - MU), where R = gc_speed / mutator_speed.
//
@@ -49,69 +49,44 @@ namespace internal {
// F = R * (1 - MU) / (R * (1 - MU) - MU)
double MemoryController::GrowingFactor(double gc_speed, double mutator_speed,
double max_factor) {
- DCHECK_LE(kMinGrowingFactor, max_factor);
- DCHECK_GE(kMaxGrowingFactor, max_factor);
+ DCHECK_LE(min_growing_factor_, max_factor);
+ DCHECK_GE(max_growing_factor_, max_factor);
if (gc_speed == 0 || mutator_speed == 0) return max_factor;
const double speed_ratio = gc_speed / mutator_speed;
- const double a = speed_ratio * (1 - kTargetMutatorUtilization);
- const double b =
- speed_ratio * (1 - kTargetMutatorUtilization) - kTargetMutatorUtilization;
+ const double a = speed_ratio * (1 - target_mutator_utilization_);
+ const double b = speed_ratio * (1 - target_mutator_utilization_) -
+ target_mutator_utilization_;
// The factor is a / b, but we need to check for small b first.
double factor = (a < b * max_factor) ? a / b : max_factor;
factor = Min(factor, max_factor);
- factor = Max(factor, kMinGrowingFactor);
- return factor;
-}
-
-double MemoryController::MaxGrowingFactor(size_t curr_max_size) {
- const double min_small_factor = 1.3;
- const double max_small_factor = 2.0;
- const double high_factor = 4.0;
-
- size_t max_size_in_mb = curr_max_size / MB;
- max_size_in_mb = Max(max_size_in_mb, kMinSize);
-
- // If we are on a device with lots of memory, we allow a high heap
- // growing factor.
- if (max_size_in_mb >= kMaxSize) {
- return high_factor;
- }
-
- DCHECK_GE(max_size_in_mb, kMinSize);
- DCHECK_LT(max_size_in_mb, kMaxSize);
-
- // On smaller devices we linearly scale the factor: (X-A)/(B-A)*(D-C)+C
- double factor = (max_size_in_mb - kMinSize) *
- (max_small_factor - min_small_factor) /
- (kMaxSize - kMinSize) +
- min_small_factor;
+ factor = Max(factor, min_growing_factor_);
return factor;
}
size_t MemoryController::CalculateAllocationLimit(
- size_t curr_size, size_t max_size, double gc_speed, double mutator_speed,
- size_t new_space_capacity, Heap::HeapGrowingMode growing_mode) {
- double max_factor = MaxGrowingFactor(max_size);
+ size_t curr_size, size_t max_size, double max_factor, double gc_speed,
+ double mutator_speed, size_t new_space_capacity,
+ Heap::HeapGrowingMode growing_mode) {
double factor = GrowingFactor(gc_speed, mutator_speed, max_factor);
if (FLAG_trace_gc_verbose) {
heap_->isolate()->PrintWithTimestamp(
"%s factor %.1f based on mu=%.3f, speed_ratio=%.f "
"(gc=%.f, mutator=%.f)\n",
- ControllerName(), factor, kTargetMutatorUtilization,
+ ControllerName(), factor, target_mutator_utilization_,
gc_speed / mutator_speed, gc_speed, mutator_speed);
}
if (growing_mode == Heap::HeapGrowingMode::kConservative ||
growing_mode == Heap::HeapGrowingMode::kSlow) {
- factor = Min(factor, kConservativeGrowingFactor);
+ factor = Min(factor, conservative_growing_factor_);
}
if (growing_mode == Heap::HeapGrowingMode::kMinimal) {
- factor = kMinGrowingFactor;
+ factor = min_growing_factor_;
}
if (FLAG_heap_growing_percent > 0) {
@@ -147,5 +122,30 @@ size_t MemoryController::MinimumAllocationLimitGrowingStep(
: kRegularAllocationLimitGrowingStep);
}
+double HeapController::MaxGrowingFactor(size_t curr_max_size) {
+ const double min_small_factor = 1.3;
+ const double max_small_factor = 2.0;
+ const double high_factor = 4.0;
+
+ size_t max_size_in_mb = curr_max_size / MB;
+ max_size_in_mb = Max(max_size_in_mb, kMinSize);
+
+ // If we are on a device with lots of memory, we allow a high heap
+ // growing factor.
+ if (max_size_in_mb >= kMaxSize) {
+ return high_factor;
+ }
+
+ DCHECK_GE(max_size_in_mb, kMinSize);
+ DCHECK_LT(max_size_in_mb, kMaxSize);
+
+ // On smaller devices we linearly scale the factor: (X-A)/(B-A)*(D-C)+C
+ double factor = (max_size_in_mb - kMinSize) *
+ (max_small_factor - min_small_factor) /
+ (kMaxSize - kMinSize) +
+ min_small_factor;
+ return factor;
+}
+
} // namespace internal
} // namespace v8
diff --git a/deps/v8/src/heap/heap-controller.h b/deps/v8/src/heap/heap-controller.h
index 8aae46c279..f8625ee963 100644
--- a/deps/v8/src/heap/heap-controller.h
+++ b/deps/v8/src/heap/heap-controller.h
@@ -18,20 +18,18 @@ class V8_EXPORT_PRIVATE MemoryController {
MemoryController(Heap* heap, double min_growing_factor,
double max_growing_factor,
double conservative_growing_factor,
- double target_mutator_utilization, size_t min_size,
- size_t max_size)
+ double target_mutator_utilization)
: heap_(heap),
- kMinGrowingFactor(min_growing_factor),
- kMaxGrowingFactor(max_growing_factor),
- kConservativeGrowingFactor(conservative_growing_factor),
- kTargetMutatorUtilization(target_mutator_utilization),
- kMinSize(min_size),
- kMaxSize(max_size) {}
- virtual ~MemoryController() {}
+ min_growing_factor_(min_growing_factor),
+ max_growing_factor_(max_growing_factor),
+ conservative_growing_factor_(conservative_growing_factor),
+ target_mutator_utilization_(target_mutator_utilization) {}
+ virtual ~MemoryController() = default;
// Computes the allocation limit to trigger the next garbage collection.
size_t CalculateAllocationLimit(size_t curr_size, size_t max_size,
- double gc_speed, double mutator_speed,
+ double max_factor, double gc_speed,
+ double mutator_speed,
size_t new_space_capacity,
Heap::HeapGrowingMode growing_mode);
@@ -41,18 +39,13 @@ class V8_EXPORT_PRIVATE MemoryController {
protected:
double GrowingFactor(double gc_speed, double mutator_speed,
double max_factor);
- double MaxGrowingFactor(size_t curr_max_size);
virtual const char* ControllerName() = 0;
Heap* const heap_;
-
- const double kMinGrowingFactor;
- const double kMaxGrowingFactor;
- const double kConservativeGrowingFactor;
- const double kTargetMutatorUtilization;
- // Sizes are in MB.
- const size_t kMinSize;
- const size_t kMaxSize;
+ const double min_growing_factor_;
+ const double max_growing_factor_;
+ const double conservative_growing_factor_;
+ const double target_mutator_utilization_;
FRIEND_TEST(HeapControllerTest, HeapGrowingFactor);
FRIEND_TEST(HeapControllerTest, MaxHeapGrowingFactor);
@@ -60,18 +53,18 @@ class V8_EXPORT_PRIVATE MemoryController {
FRIEND_TEST(HeapControllerTest, OldGenerationAllocationLimit);
};
-class HeapController : public MemoryController {
+class V8_EXPORT_PRIVATE HeapController : public MemoryController {
public:
- explicit HeapController(Heap* heap)
- : MemoryController(heap, 1.1, 4.0, 1.3, 0.97, kMinHeapSize,
- kMaxHeapSize) {}
-
// Sizes are in MB.
- static const size_t kMinHeapSize = 128 * Heap::kPointerMultiplier;
- static const size_t kMaxHeapSize = 1024 * Heap::kPointerMultiplier;
+ static constexpr size_t kMinSize = 128 * Heap::kPointerMultiplier;
+ static constexpr size_t kMaxSize = 1024 * Heap::kPointerMultiplier;
+
+ explicit HeapController(Heap* heap)
+ : MemoryController(heap, 1.1, 4.0, 1.3, 0.97) {}
+ double MaxGrowingFactor(size_t curr_max_size);
protected:
- const char* ControllerName() { return "HeapController"; }
+ const char* ControllerName() override { return "HeapController"; }
};
} // namespace internal
diff --git a/deps/v8/src/heap/heap-inl.h b/deps/v8/src/heap/heap-inl.h
index 62f07ea322..65b791a42f 100644
--- a/deps/v8/src/heap/heap-inl.h
+++ b/deps/v8/src/heap/heap-inl.h
@@ -13,6 +13,7 @@
#include "src/heap/heap-write-barrier.h"
#include "src/heap/heap.h"
+#include "src/base/atomic-utils.h"
#include "src/base/platform/platform.h"
#include "src/counters-inl.h"
#include "src/feedback-vector.h"
@@ -24,9 +25,11 @@
#include "src/log.h"
#include "src/msan.h"
#include "src/objects-inl.h"
+#include "src/objects/allocation-site-inl.h"
#include "src/objects/api-callbacks-inl.h"
#include "src/objects/descriptor-array.h"
#include "src/objects/literal-objects.h"
+#include "src/objects/microtask-queue-inl.h"
#include "src/objects/scope-info.h"
#include "src/objects/script-inl.h"
#include "src/profiler/heap-profiler.h"
@@ -52,33 +55,20 @@ HeapObject* AllocationResult::ToObjectChecked() {
return HeapObject::cast(object_);
}
-#define ROOT_ACCESSOR(type, name, camel_name) \
- type* Heap::name() { return type::cast(roots_[k##camel_name##RootIndex]); }
+#define ROOT_ACCESSOR(type, name, CamelName) \
+ type* Heap::name() { return type::cast(roots_[RootIndex::k##CamelName]); }
MUTABLE_ROOT_LIST(ROOT_ACCESSOR)
#undef ROOT_ACCESSOR
-#define DATA_HANDLER_MAP_ACCESSOR(NAME, Name, Size, name) \
- Map* Heap::name##_map() { \
- return Map::cast(roots_[k##Name##Size##MapRootIndex]); \
- }
-DATA_HANDLER_LIST(DATA_HANDLER_MAP_ACCESSOR)
-#undef DATA_HANDLER_MAP_ACCESSOR
-
-#define ACCESSOR_INFO_ACCESSOR(accessor_name, AccessorName) \
- AccessorInfo* Heap::accessor_name##_accessor() { \
- return AccessorInfo::cast(roots_[k##AccessorName##AccessorRootIndex]); \
- }
-ACCESSOR_INFO_LIST(ACCESSOR_INFO_ACCESSOR)
-#undef ACCESSOR_INFO_ACCESSOR
-
-#define ROOT_ACCESSOR(type, name, camel_name) \
- void Heap::set_##name(type* value) { \
- /* The deserializer makes use of the fact that these common roots are */ \
- /* never in new space and never on a page that is being compacted. */ \
- DCHECK(!deserialization_complete() || \
- RootCanBeWrittenAfterInitialization(k##camel_name##RootIndex)); \
- DCHECK(k##camel_name##RootIndex >= kOldSpaceRoots || !InNewSpace(value)); \
- roots_[k##camel_name##RootIndex] = value; \
+#define ROOT_ACCESSOR(type, name, CamelName) \
+ void Heap::set_##name(type* value) { \
+ /* The deserializer makes use of the fact that these common roots are */ \
+ /* never in new space and never on a page that is being compacted. */ \
+ DCHECK(!deserialization_complete() || \
+ RootCanBeWrittenAfterInitialization(RootIndex::k##CamelName)); \
+ DCHECK_IMPLIES(static_cast<int>(RootIndex::k##CamelName) < kOldSpaceRoots, \
+ !InNewSpace(value)); \
+ roots_[RootIndex::k##CamelName] = value; \
}
ROOT_LIST(ROOT_ACCESSOR)
#undef ROOT_ACCESSOR
@@ -336,8 +326,7 @@ bool Heap::InNewSpace(Object* object) {
// static
bool Heap::InNewSpace(MaybeObject* object) {
HeapObject* heap_object;
- return object->ToStrongOrWeakHeapObject(&heap_object) &&
- InNewSpace(heap_object);
+ return object->GetHeapObject(&heap_object) && InNewSpace(heap_object);
}
// static
@@ -365,8 +354,7 @@ bool Heap::InFromSpace(Object* object) {
// static
bool Heap::InFromSpace(MaybeObject* object) {
HeapObject* heap_object;
- return object->ToStrongOrWeakHeapObject(&heap_object) &&
- InFromSpace(heap_object);
+ return object->GetHeapObject(&heap_object) && InFromSpace(heap_object);
}
// static
@@ -384,8 +372,7 @@ bool Heap::InToSpace(Object* object) {
// static
bool Heap::InToSpace(MaybeObject* object) {
HeapObject* heap_object;
- return object->ToStrongOrWeakHeapObject(&heap_object) &&
- InToSpace(heap_object);
+ return object->GetHeapObject(&heap_object) && InToSpace(heap_object);
}
// static
@@ -581,6 +568,19 @@ int Heap::MaxNumberToStringCacheSize() const {
// of entries.
return static_cast<int>(number_string_cache_size * 2);
}
+
+void Heap::IncrementExternalBackingStoreBytes(ExternalBackingStoreType type,
+ size_t amount) {
+ base::CheckedIncrement(&backing_store_bytes_, amount);
+ // TODO(mlippautz): Implement interrupt for global memory allocations that can
+ // trigger garbage collections.
+}
+
+void Heap::DecrementExternalBackingStoreBytes(ExternalBackingStoreType type,
+ size_t amount) {
+ base::CheckedDecrement(&backing_store_bytes_, amount);
+}
+
AlwaysAllocateScope::AlwaysAllocateScope(Isolate* isolate)
: heap_(isolate->heap()) {
heap_->always_allocate_scope_count_++;
diff --git a/deps/v8/src/heap/heap-write-barrier-inl.h b/deps/v8/src/heap/heap-write-barrier-inl.h
index 1e4550679c..b20e65d1f1 100644
--- a/deps/v8/src/heap/heap-write-barrier-inl.h
+++ b/deps/v8/src/heap/heap-write-barrier-inl.h
@@ -96,7 +96,7 @@ inline void GenerationalBarrier(HeapObject* object, Object** slot,
inline void GenerationalBarrier(HeapObject* object, MaybeObject** slot,
MaybeObject* value) {
HeapObject* value_heap_object;
- if (!value->ToStrongOrWeakHeapObject(&value_heap_object)) return;
+ if (!value->GetHeapObject(&value_heap_object)) return;
heap_internals::GenerationalBarrierInternal(
object, reinterpret_cast<Address>(slot), value_heap_object);
}
@@ -129,7 +129,7 @@ inline void MarkingBarrier(HeapObject* object, Object** slot, Object* value) {
inline void MarkingBarrier(HeapObject* object, MaybeObject** slot,
MaybeObject* value) {
HeapObject* value_heap_object;
- if (!value->ToStrongOrWeakHeapObject(&value_heap_object)) return;
+ if (!value->GetHeapObject(&value_heap_object)) return;
heap_internals::MarkingBarrierInternal(
object, reinterpret_cast<Address>(slot), value_heap_object);
}
diff --git a/deps/v8/src/heap/heap.cc b/deps/v8/src/heap/heap.cc
index 2ec30635be..b509d21142 100644
--- a/deps/v8/src/heap/heap.cc
+++ b/deps/v8/src/heap/heap.cc
@@ -33,7 +33,6 @@
#include "src/heap/heap-controller.h"
#include "src/heap/heap-write-barrier-inl.h"
#include "src/heap/incremental-marking.h"
-#include "src/heap/item-parallel-job.h"
#include "src/heap/mark-compact-inl.h"
#include "src/heap/mark-compact.h"
#include "src/heap/memory-reducer.h"
@@ -108,12 +107,7 @@ bool Heap::GCCallbackTuple::operator==(
}
Heap::GCCallbackTuple& Heap::GCCallbackTuple::operator=(
- const Heap::GCCallbackTuple& other) {
- callback = other.callback;
- gc_type = other.gc_type;
- data = other.data;
- return *this;
-}
+ const Heap::GCCallbackTuple& other) = default;
struct Heap::StrongRootsList {
Object** start;
@@ -135,112 +129,17 @@ class IdleScavengeObserver : public AllocationObserver {
};
Heap::Heap()
- : external_memory_(0),
- external_memory_limit_(kExternalAllocationSoftLimit),
- external_memory_at_last_mark_compact_(0),
- external_memory_concurrently_freed_(0),
- isolate_(nullptr),
- code_range_size_(0),
- // semispace_size_ should be a power of 2 and old_generation_size_ should
- // be a multiple of Page::kPageSize.
- max_semi_space_size_(8 * (kPointerSize / 4) * MB),
- initial_semispace_size_(kMinSemiSpaceSizeInKB * KB),
- max_old_generation_size_(700ul * (kPointerSize / 4) * MB),
- initial_max_old_generation_size_(max_old_generation_size_),
+ : initial_max_old_generation_size_(max_old_generation_size_),
initial_old_generation_size_(max_old_generation_size_ /
kInitalOldGenerationLimitFactor),
- old_generation_size_configured_(false),
- // Variables set based on semispace_size_ and old_generation_size_ in
- // ConfigureHeap.
- // Will be 4 * reserved_semispace_size_ to ensure that young
- // generation can be aligned to its size.
- maximum_committed_(0),
- survived_since_last_expansion_(0),
- survived_last_scavenge_(0),
- always_allocate_scope_count_(0),
memory_pressure_level_(MemoryPressureLevel::kNone),
- contexts_disposed_(0),
- number_of_disposed_maps_(0),
- new_space_(nullptr),
- old_space_(nullptr),
- code_space_(nullptr),
- map_space_(nullptr),
- lo_space_(nullptr),
- new_lo_space_(nullptr),
- read_only_space_(nullptr),
- write_protect_code_memory_(false),
- code_space_memory_modification_scope_depth_(0),
- gc_state_(NOT_IN_GC),
- gc_post_processing_depth_(0),
- allocations_count_(0),
- raw_allocations_hash_(0),
- stress_marking_observer_(nullptr),
- stress_scavenge_observer_(nullptr),
- allocation_step_in_progress_(false),
- max_marking_limit_reached_(0.0),
- ms_count_(0),
- gc_count_(0),
- consecutive_ineffective_mark_compacts_(0),
- mmap_region_base_(0),
- remembered_unmapped_pages_index_(0),
old_generation_allocation_limit_(initial_old_generation_size_),
- inline_allocation_disabled_(false),
- tracer_(nullptr),
- promoted_objects_size_(0),
- promotion_ratio_(0),
- semi_space_copied_object_size_(0),
- previous_semi_space_copied_object_size_(0),
- semi_space_copied_rate_(0),
- nodes_died_in_new_space_(0),
- nodes_copied_in_new_space_(0),
- nodes_promoted_(0),
- maximum_size_scavenges_(0),
- last_idle_notification_time_(0.0),
- last_gc_time_(0.0),
- mark_compact_collector_(nullptr),
- minor_mark_compact_collector_(nullptr),
- array_buffer_collector_(nullptr),
- memory_allocator_(nullptr),
- store_buffer_(nullptr),
- incremental_marking_(nullptr),
- concurrent_marking_(nullptr),
- gc_idle_time_handler_(nullptr),
- memory_reducer_(nullptr),
- live_object_stats_(nullptr),
- dead_object_stats_(nullptr),
- scavenge_job_(nullptr),
- parallel_scavenge_semaphore_(0),
- idle_scavenge_observer_(nullptr),
- new_space_allocation_counter_(0),
- old_generation_allocation_counter_at_last_gc_(0),
- old_generation_size_at_last_gc_(0),
global_pretenuring_feedback_(kInitialFeedbackCapacity),
- is_marking_flag_(false),
- ring_buffer_full_(false),
- ring_buffer_end_(0),
- configured_(false),
- current_gc_flags_(Heap::kNoGCFlags),
current_gc_callback_flags_(GCCallbackFlags::kNoGCCallbackFlags),
- external_string_table_(this),
- gc_callbacks_depth_(0),
- deserialization_complete_(false),
- strong_roots_list_(nullptr),
- heap_iterator_depth_(0),
- local_embedder_heap_tracer_(nullptr),
- fast_promotion_mode_(false),
- force_oom_(false),
- delay_sweeper_tasks_for_testing_(false),
- pending_layout_change_object_(nullptr),
- unprotected_memory_chunks_registry_enabled_(false)
-#ifdef V8_ENABLE_ALLOCATION_TIMEOUT
- ,
- allocation_timeout_(0)
-#endif // V8_ENABLE_ALLOCATION_TIMEOUT
-{
+ external_string_table_(this) {
// Ensure old_generation_size_ is a multiple of kPageSize.
DCHECK_EQ(0, max_old_generation_size_ & (Page::kPageSize - 1));
- memset(roots_, 0, sizeof(roots_[0]) * kRootListLength);
set_native_contexts_list(nullptr);
set_allocation_sites_list(Smi::kZero);
// Put a dummy entry in the remembered pages so we can find the list the
@@ -259,8 +158,8 @@ size_t Heap::ComputeMaxOldGenerationSize(uint64_t physical_memory) {
size_t computed_size = static_cast<size_t>(physical_memory / i::MB /
old_space_physical_memory_factor *
kPointerMultiplier);
- return Max(Min(computed_size, HeapController::kMaxHeapSize),
- HeapController::kMinHeapSize);
+ return Max(Min(computed_size, HeapController::kMaxSize),
+ HeapController::kMinSize);
}
size_t Heap::Capacity() {
@@ -477,6 +376,8 @@ void Heap::PrintShortHeapStatistics() {
CommittedMemoryOfHeapAndUnmapper() / KB);
PrintIsolate(isolate_, "External memory reported: %6" PRId64 " KB\n",
external_memory_ / KB);
+ PrintIsolate(isolate_, "Backing store memory: %6" PRIuS " KB\n",
+ backing_store_bytes_ / KB);
PrintIsolate(isolate_, "External memory global %zu KB\n",
external_memory_callback_() / KB);
PrintIsolate(isolate_, "Total time spent in GC : %.1f ms\n",
@@ -531,7 +432,7 @@ bool Heap::IsRetainingPathTarget(HeapObject* object,
MaybeObject* object_to_check = HeapObjectReference::Weak(object);
for (int i = 0; i < length; i++) {
MaybeObject* target = targets->Get(i);
- DCHECK(target->IsWeakOrClearedHeapObject());
+ DCHECK(target->IsWeakOrCleared());
if (target == object_to_check) {
DCHECK(retaining_path_target_option_.count(i));
*option = retaining_path_target_option_[i];
@@ -705,7 +606,7 @@ const char* Heap::GetSpaceName(int idx) {
}
void Heap::SetRootCodeStubs(SimpleNumberDictionary* value) {
- roots_[kCodeStubsRootIndex] = value;
+ roots_[RootIndex::kCodeStubs] = value;
}
void Heap::RepairFreeListsAfterDeserialization() {
@@ -1219,13 +1120,16 @@ void Heap::CollectAllAvailableGarbage(GarbageCollectionReason gc_reason) {
// The optimizing compiler may be unnecessarily holding on to memory.
isolate()->AbortConcurrentOptimization(BlockingBehavior::kDontBlock);
isolate()->ClearSerializerData();
- set_current_gc_flags(kMakeHeapIterableMask | kReduceMemoryFootprintMask);
+ set_current_gc_flags(kReduceMemoryFootprintMask);
isolate_->compilation_cache()->Clear();
const int kMaxNumberOfAttempts = 7;
const int kMinNumberOfAttempts = 2;
+ const v8::GCCallbackFlags callback_flags =
+ gc_reason == GarbageCollectionReason::kLowMemoryNotification
+ ? v8::kGCCallbackFlagForced
+ : v8::kGCCallbackFlagCollectAllAvailableGarbage;
for (int attempt = 0; attempt < kMaxNumberOfAttempts; attempt++) {
- if (!CollectGarbage(OLD_SPACE, gc_reason,
- v8::kGCCallbackFlagCollectAllAvailableGarbage) &&
+ if (!CollectGarbage(OLD_SPACE, gc_reason, callback_flags) &&
attempt + 1 >= kMinNumberOfAttempts) {
break;
}
@@ -1234,7 +1138,7 @@ void Heap::CollectAllAvailableGarbage(GarbageCollectionReason gc_reason) {
set_current_gc_flags(kNoGCFlags);
new_space_->Shrink();
UncommitFromSpace();
- memory_allocator()->unmapper()->EnsureUnmappingCompleted();
+ EagerlyFreeExternalMemory();
if (FLAG_trace_duplicate_threshold_kb) {
std::map<int, std::vector<HeapObject*>> objects_by_size;
@@ -1259,6 +1163,15 @@ void Heap::CollectAllAvailableGarbage(GarbageCollectionReason gc_reason) {
}
}
+void Heap::PreciseCollectAllGarbage(int flags,
+ GarbageCollectionReason gc_reason,
+ const GCCallbackFlags gc_callback_flags) {
+ if (!incremental_marking()->IsStopped()) {
+ FinalizeIncrementalMarkingAtomically(gc_reason);
+ }
+ CollectAllGarbage(flags, gc_reason, gc_callback_flags);
+}
+
void Heap::ReportExternalMemoryPressure() {
const GCCallbackFlags kGCCallbackFlagsForExternalMemory =
static_cast<GCCallbackFlags>(
@@ -1267,7 +1180,7 @@ void Heap::ReportExternalMemoryPressure() {
if (external_memory_ >
(external_memory_at_last_mark_compact_ + external_memory_hard_limit())) {
CollectAllGarbage(
- kReduceMemoryFootprintMask | kFinalizeIncrementalMarkingMask,
+ kReduceMemoryFootprintMask,
GarbageCollectionReason::kExternalMemoryPressure,
static_cast<GCCallbackFlags>(kGCCallbackFlagCollectAllAvailableGarbage |
kGCCallbackFlagsForExternalMemory));
@@ -1377,8 +1290,8 @@ bool Heap::CollectGarbage(AllocationSpace space,
next_gc_likely_to_collect_more =
PerformGarbageCollection(collector, gc_callback_flags);
- if (collector == MARK_COMPACTOR) {
- tracer()->RecordMarkCompactHistograms(gc_type_timer);
+ if (collector == MARK_COMPACTOR || collector == SCAVENGER) {
+ tracer()->RecordGCPhasesHistograms(gc_type_timer);
}
}
@@ -1416,12 +1329,9 @@ bool Heap::CollectGarbage(AllocationSpace space,
isolate()->CountUsage(v8::Isolate::kForcedGC);
}
- // Start incremental marking for the next cycle. The heap snapshot
- // generator needs incremental marking to stay off after it aborted.
- // We do this only for scavenger to avoid a loop where mark-compact
- // causes another mark-compact.
- if (IsYoungGenerationCollector(collector) &&
- !ShouldAbortIncrementalMarking()) {
+ // Start incremental marking for the next cycle. We do this only for scavenger
+ // to avoid a loop where mark-compact causes another mark-compact.
+ if (IsYoungGenerationCollector(collector)) {
StartIncrementalMarkingIfAllocationLimitIsReached(
GCFlagsForIncrementalMarking(),
kGCCallbackScheduleIdleGarbageCollection);
@@ -1627,11 +1537,10 @@ bool Heap::ReserveSpace(Reservation* reservations, std::vector<Address>* maps) {
CollectGarbage(NEW_SPACE, GarbageCollectionReason::kDeserializer);
} else {
if (counter > 1) {
- CollectAllGarbage(
- kReduceMemoryFootprintMask | kAbortIncrementalMarkingMask,
- GarbageCollectionReason::kDeserializer);
+ CollectAllGarbage(kReduceMemoryFootprintMask,
+ GarbageCollectionReason::kDeserializer);
} else {
- CollectAllGarbage(kAbortIncrementalMarkingMask,
+ CollectAllGarbage(kNoGCFlags,
GarbageCollectionReason::kDeserializer);
}
}
@@ -1785,18 +1694,22 @@ bool Heap::PerformGarbageCollection(
external_memory_at_last_mark_compact_ = external_memory_;
external_memory_limit_ = external_memory_ + kExternalAllocationSoftLimit;
+ double max_factor =
+ heap_controller()->MaxGrowingFactor(max_old_generation_size_);
size_t new_limit = heap_controller()->CalculateAllocationLimit(
- old_gen_size, max_old_generation_size_, gc_speed, mutator_speed,
- new_space()->Capacity(), CurrentHeapGrowingMode());
+ old_gen_size, max_old_generation_size_, max_factor, gc_speed,
+ mutator_speed, new_space()->Capacity(), CurrentHeapGrowingMode());
old_generation_allocation_limit_ = new_limit;
CheckIneffectiveMarkCompact(
old_gen_size, tracer()->AverageMarkCompactMutatorUtilization());
} else if (HasLowYoungGenerationAllocationRate() &&
old_generation_size_configured_) {
+ double max_factor =
+ heap_controller()->MaxGrowingFactor(max_old_generation_size_);
size_t new_limit = heap_controller()->CalculateAllocationLimit(
- old_gen_size, max_old_generation_size_, gc_speed, mutator_speed,
- new_space()->Capacity(), CurrentHeapGrowingMode());
+ old_gen_size, max_old_generation_size_, max_factor, gc_speed,
+ mutator_speed, new_space()->Capacity(), CurrentHeapGrowingMode());
if (new_limit < old_generation_allocation_limit_) {
old_generation_allocation_limit_ = new_limit;
}
@@ -1940,26 +1853,6 @@ void Heap::CheckNewSpaceExpansionCriteria() {
}
}
-static bool IsUnscavengedHeapObject(Heap* heap, Object** p) {
- return Heap::InFromSpace(*p) &&
- !HeapObject::cast(*p)->map_word().IsForwardingAddress();
-}
-
-class ScavengeWeakObjectRetainer : public WeakObjectRetainer {
- public:
- virtual Object* RetainAs(Object* object) {
- if (!Heap::InFromSpace(object)) {
- return object;
- }
-
- MapWord map_word = HeapObject::cast(object)->map_word();
- if (map_word.IsForwardingAddress()) {
- return map_word.ToForwardingAddress();
- }
- return nullptr;
- }
-};
-
void Heap::EvacuateYoungGeneration() {
TRACE_GC(tracer(), GCTracer::Scope::SCAVENGER_FAST_PROMOTE);
base::LockGuard<base::Mutex> guard(relocation_mutex());
@@ -2003,79 +1896,6 @@ void Heap::EvacuateYoungGeneration() {
SetGCState(NOT_IN_GC);
}
-static bool IsLogging(Isolate* isolate) {
- return FLAG_verify_predictable || isolate->logger()->is_logging() ||
- isolate->is_profiling() ||
- (isolate->heap_profiler() != nullptr &&
- isolate->heap_profiler()->is_tracking_object_moves()) ||
- isolate->heap()->has_heap_object_allocation_tracker();
-}
-
-class PageScavengingItem final : public ItemParallelJob::Item {
- public:
- explicit PageScavengingItem(MemoryChunk* chunk) : chunk_(chunk) {}
- virtual ~PageScavengingItem() {}
-
- void Process(Scavenger* scavenger) { scavenger->ScavengePage(chunk_); }
-
- private:
- MemoryChunk* const chunk_;
-};
-
-class ScavengingTask final : public ItemParallelJob::Task {
- public:
- ScavengingTask(Heap* heap, Scavenger* scavenger, OneshotBarrier* barrier)
- : ItemParallelJob::Task(heap->isolate()),
- heap_(heap),
- scavenger_(scavenger),
- barrier_(barrier) {}
-
- void RunInParallel() final {
- TRACE_BACKGROUND_GC(
- heap_->tracer(),
- GCTracer::BackgroundScope::SCAVENGER_BACKGROUND_SCAVENGE_PARALLEL);
- double scavenging_time = 0.0;
- {
- barrier_->Start();
- TimedScope scope(&scavenging_time);
- PageScavengingItem* item = nullptr;
- while ((item = GetItem<PageScavengingItem>()) != nullptr) {
- item->Process(scavenger_);
- item->MarkFinished();
- }
- do {
- scavenger_->Process(barrier_);
- } while (!barrier_->Wait());
- scavenger_->Process();
- }
- if (FLAG_trace_parallel_scavenge) {
- PrintIsolate(heap_->isolate(),
- "scavenge[%p]: time=%.2f copied=%zu promoted=%zu\n",
- static_cast<void*>(this), scavenging_time,
- scavenger_->bytes_copied(), scavenger_->bytes_promoted());
- }
- };
-
- private:
- Heap* const heap_;
- Scavenger* const scavenger_;
- OneshotBarrier* const barrier_;
-};
-
-int Heap::NumberOfScavengeTasks() {
- if (!FLAG_parallel_scavenge) return 1;
- const int num_scavenge_tasks =
- static_cast<int>(new_space()->TotalCapacity()) / MB;
- static int num_cores = V8::GetCurrentPlatform()->NumberOfWorkerThreads() + 1;
- int tasks =
- Max(1, Min(Min(num_scavenge_tasks, kMaxScavengerTasks), num_cores));
- if (!CanExpandOldGeneration(static_cast<size_t>(tasks * Page::kPageSize))) {
- // Optimize for memory usage near the heap limit.
- tasks = 1;
- }
- return tasks;
-}
-
void Heap::Scavenge() {
TRACE_GC(tracer(), GCTracer::Scope::SCAVENGER_SCAVENGE);
base::LockGuard<base::Mutex> guard(relocation_mutex());
@@ -2088,7 +1908,6 @@ void Heap::Scavenge() {
// Bump-pointer allocations done during scavenge are not real allocations.
// Pause the inline allocation steps.
PauseAllocationObserversScope pause_observers(this);
-
IncrementalMarking::PauseBlackAllocationScope pause_black_allocation(
incremental_marking());
@@ -2097,137 +1916,19 @@ void Heap::Scavenge() {
SetGCState(SCAVENGE);
- // Implements Cheney's copying algorithm
- LOG(isolate_, ResourceEvent("scavenge", "begin"));
-
// Flip the semispaces. After flipping, to space is empty, from space has
// live objects.
- new_space_->Flip();
- new_space_->ResetLinearAllocationArea();
-
- ItemParallelJob job(isolate()->cancelable_task_manager(),
- &parallel_scavenge_semaphore_);
- const int kMainThreadId = 0;
- Scavenger* scavengers[kMaxScavengerTasks];
- const bool is_logging = IsLogging(isolate());
- const int num_scavenge_tasks = NumberOfScavengeTasks();
- OneshotBarrier barrier;
- Scavenger::CopiedList copied_list(num_scavenge_tasks);
- Scavenger::PromotionList promotion_list(num_scavenge_tasks);
- for (int i = 0; i < num_scavenge_tasks; i++) {
- scavengers[i] =
- new Scavenger(this, is_logging, &copied_list, &promotion_list, i);
- job.AddTask(new ScavengingTask(this, scavengers[i], &barrier));
- }
-
- {
- Sweeper* sweeper = mark_compact_collector()->sweeper();
- // Pause the concurrent sweeper.
- Sweeper::PauseOrCompleteScope pause_scope(sweeper);
- // Filter out pages from the sweeper that need to be processed for old to
- // new slots by the Scavenger. After processing, the Scavenger adds back
- // pages that are still unsweeped. This way the Scavenger has exclusive
- // access to the slots of a page and can completely avoid any locks on
- // the page itself.
- Sweeper::FilterSweepingPagesScope filter_scope(sweeper, pause_scope);
- filter_scope.FilterOldSpaceSweepingPages(
- [](Page* page) { return !page->ContainsSlots<OLD_TO_NEW>(); });
- RememberedSet<OLD_TO_NEW>::IterateMemoryChunks(
- this, [&job](MemoryChunk* chunk) {
- job.AddItem(new PageScavengingItem(chunk));
- });
-
- RootScavengeVisitor root_scavenge_visitor(scavengers[kMainThreadId]);
-
- {
- // Identify weak unmodified handles. Requires an unmodified graph.
- TRACE_GC(
- tracer(),
- GCTracer::Scope::SCAVENGER_SCAVENGE_WEAK_GLOBAL_HANDLES_IDENTIFY);
- isolate()->global_handles()->IdentifyWeakUnmodifiedObjects(
- &JSObject::IsUnmodifiedApiObject);
- }
- {
- // Copy roots.
- TRACE_GC(tracer(), GCTracer::Scope::SCAVENGER_SCAVENGE_ROOTS);
- IterateRoots(&root_scavenge_visitor, VISIT_ALL_IN_SCAVENGE);
- }
- {
- // Parallel phase scavenging all copied and promoted objects.
- TRACE_GC(tracer(), GCTracer::Scope::SCAVENGER_SCAVENGE_PARALLEL);
- job.Run(isolate()->async_counters());
- DCHECK(copied_list.IsEmpty());
- DCHECK(promotion_list.IsEmpty());
- }
- {
- // Scavenge weak global handles.
- TRACE_GC(tracer(),
- GCTracer::Scope::SCAVENGER_SCAVENGE_WEAK_GLOBAL_HANDLES_PROCESS);
- isolate()->global_handles()->MarkNewSpaceWeakUnmodifiedObjectsPending(
- &IsUnscavengedHeapObject);
- isolate()
- ->global_handles()
- ->IterateNewSpaceWeakUnmodifiedRootsForFinalizers(
- &root_scavenge_visitor);
- scavengers[kMainThreadId]->Process();
-
- DCHECK(copied_list.IsEmpty());
- DCHECK(promotion_list.IsEmpty());
- isolate()
- ->global_handles()
- ->IterateNewSpaceWeakUnmodifiedRootsForPhantomHandles(
- &root_scavenge_visitor, &IsUnscavengedHeapObject);
- }
-
- for (int i = 0; i < num_scavenge_tasks; i++) {
- scavengers[i]->Finalize();
- delete scavengers[i];
- }
- }
-
- {
- // Update references into new space
- TRACE_GC(tracer(), GCTracer::Scope::SCAVENGER_SCAVENGE_UPDATE_REFS);
- UpdateNewSpaceReferencesInExternalStringTable(
- &UpdateNewSpaceReferenceInExternalStringTableEntry);
-
- incremental_marking()->UpdateMarkingWorklistAfterScavenge();
- }
-
- if (FLAG_concurrent_marking) {
- // Ensure that concurrent marker does not track pages that are
- // going to be unmapped.
- for (Page* p : PageRange(new_space()->from_space().first_page(), nullptr)) {
- concurrent_marking()->ClearLiveness(p);
- }
- }
-
- ScavengeWeakObjectRetainer weak_object_retainer;
- ProcessYoungWeakReferences(&weak_object_retainer);
-
- // Set age mark.
- new_space_->set_age_mark(new_space_->top());
-
- {
- TRACE_GC(tracer(), GCTracer::Scope::SCAVENGER_PROCESS_ARRAY_BUFFERS);
- ArrayBufferTracker::PrepareToFreeDeadInNewSpace(this);
- }
- array_buffer_collector()->FreeAllocationsOnBackgroundThread();
+ new_space()->Flip();
+ new_space()->ResetLinearAllocationArea();
- RememberedSet<OLD_TO_NEW>::IterateMemoryChunks(this, [](MemoryChunk* chunk) {
- if (chunk->SweepingDone()) {
- RememberedSet<OLD_TO_NEW>::FreeEmptyBuckets(chunk);
- } else {
- RememberedSet<OLD_TO_NEW>::PreFreeEmptyBuckets(chunk);
- }
- });
+ // We also flip the young generation large object space. All large objects
+ // will be in the from space.
+ new_lo_space()->Flip();
- // Update how much has survived scavenge.
- IncrementYoungSurvivorsCounter(SurvivedNewSpaceObjectSize());
+ // Implements Cheney's copying algorithm
+ LOG(isolate_, ResourceEvent("scavenge", "begin"));
- // Scavenger may find new wrappers by iterating objects promoted onto a black
- // page.
- local_embedder_heap_tracer()->RegisterWrappersWithRemoteTracer();
+ scavenger_collector_->CollectGarbage();
LOG(isolate_, ResourceEvent("scavenge", "end"));
@@ -2285,15 +1986,6 @@ bool Heap::ExternalStringTable::Contains(HeapObject* obj) {
return false;
}
-void Heap::ProcessMovedExternalString(Page* old_page, Page* new_page,
- ExternalString* string) {
- size_t size = string->ExternalPayloadSize();
- new_page->IncrementExternalBackingStoreBytes(
- ExternalBackingStoreType::kExternalString, size);
- old_page->DecrementExternalBackingStoreBytes(
- ExternalBackingStoreType::kExternalString, size);
-}
-
String* Heap::UpdateNewSpaceReferenceInExternalStringTableEntry(Heap* heap,
Object** p) {
MapWord first_word = HeapObject::cast(*p)->map_word();
@@ -2312,18 +2004,15 @@ String* Heap::UpdateNewSpaceReferenceInExternalStringTableEntry(Heap* heap,
// String is still reachable.
String* new_string = String::cast(first_word.ToForwardingAddress());
- String* original_string = reinterpret_cast<String*>(*p);
- // The length of the original string is used to disambiguate the scenario
- // of a ThingString being forwarded to an ExternalString (which already exists
- // in the OLD space), and an ExternalString being forwarded to its promoted
- // copy. See Scavenger::EvacuateThinString.
- if (new_string->IsThinString() || original_string->length() == 0) {
+ if (new_string->IsThinString()) {
// Filtering Thin strings out of the external string table.
return nullptr;
} else if (new_string->IsExternalString()) {
- heap->ProcessMovedExternalString(
+ MemoryChunk::MoveExternalBackingStoreBytes(
+ ExternalBackingStoreType::kExternalString,
Page::FromAddress(reinterpret_cast<Address>(*p)),
- Page::FromHeapObject(new_string), ExternalString::cast(new_string));
+ Page::FromHeapObject(new_string),
+ ExternalString::cast(new_string)->ExternalPayloadSize());
return new_string;
}
@@ -2488,8 +2177,8 @@ void Heap::ProcessWeakListRoots(WeakObjectRetainer* retainer) {
set_allocation_sites_list(retainer->RetainAs(allocation_sites_list()));
}
-void Heap::ForeachAllocationSite(Object* list,
- std::function<void(AllocationSite*)> visitor) {
+void Heap::ForeachAllocationSite(
+ Object* list, const std::function<void(AllocationSite*)>& visitor) {
DisallowHeapAllocation disallow_heap_allocation;
Object* current = list;
while (current->IsAllocationSite()) {
@@ -2555,8 +2244,8 @@ void Heap::VisitExternalResources(v8::ExternalResourceVisitor* visitor) {
explicit ExternalStringTableVisitorAdapter(
Isolate* isolate, v8::ExternalResourceVisitor* visitor)
: isolate_(isolate), visitor_(visitor) {}
- virtual void VisitRootPointers(Root root, const char* description,
- Object** start, Object** end) {
+ void VisitRootPointers(Root root, const char* description,
+ Object** start, Object** end) override {
for (Object** p = start; p < end; p++) {
DCHECK((*p)->IsExternalString());
visitor_->VisitExternalString(
@@ -2597,10 +2286,9 @@ int Heap::GetMaximumFillToAlign(AllocationAlignment alignment) {
int Heap::GetFillToAlign(Address address, AllocationAlignment alignment) {
- intptr_t offset = OffsetFrom(address);
- if (alignment == kDoubleAligned && (offset & kDoubleAlignmentMask) != 0)
+ if (alignment == kDoubleAligned && (address & kDoubleAlignmentMask) != 0)
return kPointerSize;
- if (alignment == kDoubleUnaligned && (offset & kDoubleAlignmentMask) == 0)
+ if (alignment == kDoubleUnaligned && (address & kDoubleAlignmentMask) == 0)
return kDoubleSize - kPointerSize; // No fill if double is always aligned.
return 0;
}
@@ -2693,33 +2381,29 @@ void Heap::CreateFixedStubs() {
Heap::CreateJSRunMicrotasksEntryStub();
}
-bool Heap::RootCanBeWrittenAfterInitialization(Heap::RootListIndex root_index) {
+bool Heap::RootCanBeWrittenAfterInitialization(RootIndex root_index) {
switch (root_index) {
- case kNumberStringCacheRootIndex:
- case kCodeStubsRootIndex:
- case kScriptListRootIndex:
- case kMaterializedObjectsRootIndex:
- case kMicrotaskQueueRootIndex:
- case kDetachedContextsRootIndex:
- case kRetainedMapsRootIndex:
- case kRetainingPathTargetsRootIndex:
- case kFeedbackVectorsForProfilingToolsRootIndex:
- case kNoScriptSharedFunctionInfosRootIndex:
- case kSerializedObjectsRootIndex:
- case kSerializedGlobalProxySizesRootIndex:
- case kPublicSymbolTableRootIndex:
- case kApiSymbolTableRootIndex:
- case kApiPrivateSymbolTableRootIndex:
- case kMessageListenersRootIndex:
- case kDeserializeLazyHandlerRootIndex:
- case kDeserializeLazyHandlerWideRootIndex:
- case kDeserializeLazyHandlerExtraWideRootIndex:
+ case RootIndex::kNumberStringCache:
+ case RootIndex::kCodeStubs:
+ case RootIndex::kScriptList:
+ case RootIndex::kMaterializedObjects:
+ case RootIndex::kDetachedContexts:
+ case RootIndex::kRetainedMaps:
+ case RootIndex::kRetainingPathTargets:
+ case RootIndex::kFeedbackVectorsForProfilingTools:
+ case RootIndex::kNoScriptSharedFunctionInfos:
+ case RootIndex::kSerializedObjects:
+ case RootIndex::kSerializedGlobalProxySizes:
+ case RootIndex::kPublicSymbolTable:
+ case RootIndex::kApiSymbolTable:
+ case RootIndex::kApiPrivateSymbolTable:
+ case RootIndex::kMessageListeners:
// Smi values
-#define SMI_ENTRY(type, name, Name) case k##Name##RootIndex:
+#define SMI_ENTRY(type, name, Name) case RootIndex::k##Name:
SMI_ROOT_LIST(SMI_ENTRY)
#undef SMI_ENTRY
// String table
- case kStringTableRootIndex:
+ case RootIndex::kStringTable:
return true;
default:
@@ -2727,7 +2411,7 @@ bool Heap::RootCanBeWrittenAfterInitialization(Heap::RootListIndex root_index) {
}
}
-bool Heap::RootCanBeTreatedAsConstant(RootListIndex root_index) {
+bool Heap::RootCanBeTreatedAsConstant(RootIndex root_index) {
bool can_be = !RootCanBeWrittenAfterInitialization(root_index) &&
!InNewSpace(root(root_index));
DCHECK_IMPLIES(can_be, IsImmovable(HeapObject::cast(root(root_index))));
@@ -2743,61 +2427,6 @@ void Heap::FlushNumberStringCache() {
}
}
-namespace {
-
-Heap::RootListIndex RootIndexForFixedTypedArray(ExternalArrayType array_type) {
- switch (array_type) {
-#define ARRAY_TYPE_TO_ROOT_INDEX(Type, type, TYPE, ctype) \
- case kExternal##Type##Array: \
- return Heap::kFixed##Type##ArrayMapRootIndex;
-
- TYPED_ARRAYS(ARRAY_TYPE_TO_ROOT_INDEX)
-#undef ARRAY_TYPE_TO_ROOT_INDEX
- }
- UNREACHABLE();
-}
-
-Heap::RootListIndex RootIndexForFixedTypedArray(ElementsKind elements_kind) {
- switch (elements_kind) {
-#define TYPED_ARRAY_CASE(Type, type, TYPE, ctype) \
- case TYPE##_ELEMENTS: \
- return Heap::kFixed##Type##ArrayMapRootIndex;
- TYPED_ARRAYS(TYPED_ARRAY_CASE)
- default:
- UNREACHABLE();
-#undef TYPED_ARRAY_CASE
- }
-}
-
-Heap::RootListIndex RootIndexForEmptyFixedTypedArray(
- ElementsKind elements_kind) {
- switch (elements_kind) {
-#define ELEMENT_KIND_TO_ROOT_INDEX(Type, type, TYPE, ctype) \
- case TYPE##_ELEMENTS: \
- return Heap::kEmptyFixed##Type##ArrayRootIndex;
-
- TYPED_ARRAYS(ELEMENT_KIND_TO_ROOT_INDEX)
-#undef ELEMENT_KIND_TO_ROOT_INDEX
- default:
- UNREACHABLE();
- }
-}
-
-} // namespace
-
-Map* Heap::MapForFixedTypedArray(ExternalArrayType array_type) {
- return Map::cast(roots_[RootIndexForFixedTypedArray(array_type)]);
-}
-
-Map* Heap::MapForFixedTypedArray(ElementsKind elements_kind) {
- return Map::cast(roots_[RootIndexForFixedTypedArray(elements_kind)]);
-}
-
-FixedTypedArrayBase* Heap::EmptyFixedTypedArrayForMap(const Map* map) {
- return FixedTypedArrayBase::cast(
- roots_[RootIndexForEmptyFixedTypedArray(map->elements_kind())]);
-}
-
HeapObject* Heap::CreateFillerObjectAt(Address addr, int size,
ClearRecordedSlots clear_slots_mode,
ClearFreedMemoryMode clear_memory_mode) {
@@ -2805,11 +2434,11 @@ HeapObject* Heap::CreateFillerObjectAt(Address addr, int size,
HeapObject* filler = HeapObject::FromAddress(addr);
if (size == kPointerSize) {
filler->set_map_after_allocation(
- reinterpret_cast<Map*>(root(kOnePointerFillerMapRootIndex)),
+ reinterpret_cast<Map*>(root(RootIndex::kOnePointerFillerMap)),
SKIP_WRITE_BARRIER);
} else if (size == 2 * kPointerSize) {
filler->set_map_after_allocation(
- reinterpret_cast<Map*>(root(kTwoPointerFillerMapRootIndex)),
+ reinterpret_cast<Map*>(root(RootIndex::kTwoPointerFillerMap)),
SKIP_WRITE_BARRIER);
if (clear_memory_mode == ClearFreedMemoryMode::kClearFreedMemory) {
Memory<Address>(addr + kPointerSize) =
@@ -2818,7 +2447,7 @@ HeapObject* Heap::CreateFillerObjectAt(Address addr, int size,
} else {
DCHECK_GT(size, 2 * kPointerSize);
filler->set_map_after_allocation(
- reinterpret_cast<Map*>(root(kFreeSpaceMapRootIndex)),
+ reinterpret_cast<Map*>(root(RootIndex::kFreeSpaceMap)),
SKIP_WRITE_BARRIER);
FreeSpace::cast(filler)->relaxed_write_size(size);
if (clear_memory_mode == ClearFreedMemoryMode::kClearFreedMemory) {
@@ -2865,8 +2494,8 @@ class LeftTrimmerVerifierRootVisitor : public RootVisitor {
explicit LeftTrimmerVerifierRootVisitor(FixedArrayBase* to_check)
: to_check_(to_check) {}
- virtual void VisitRootPointers(Root root, const char* description,
- Object** start, Object** end) {
+ void VisitRootPointers(Root root, const char* description,
+ Object** start, Object** end) override {
for (Object** p = start; p < end; ++p) {
DCHECK_NE(*p, to_check_);
}
@@ -3532,7 +3161,7 @@ class MemoryPressureInterruptTask : public CancelableTask {
explicit MemoryPressureInterruptTask(Heap* heap)
: CancelableTask(heap->isolate()), heap_(heap) {}
- virtual ~MemoryPressureInterruptTask() {}
+ ~MemoryPressureInterruptTask() override = default;
private:
// v8::internal::CancelableTask overrides.
@@ -3575,9 +3204,10 @@ void Heap::CollectGarbageOnMemoryPressure() {
const double kMaxMemoryPressurePauseMs = 100;
double start = MonotonicallyIncreasingTimeInMs();
- CollectAllGarbage(kReduceMemoryFootprintMask | kAbortIncrementalMarkingMask,
+ CollectAllGarbage(kReduceMemoryFootprintMask,
GarbageCollectionReason::kMemoryPressure,
kGCCallbackFlagCollectAllAvailableGarbage);
+ EagerlyFreeExternalMemory();
double end = MonotonicallyIncreasingTimeInMs();
// Estimate how much memory we can free.
@@ -3591,10 +3221,9 @@ void Heap::CollectGarbageOnMemoryPressure() {
// If we spent less than half of the time budget, then perform full GC
// Otherwise, start incremental marking.
if (end - start < kMaxMemoryPressurePauseMs / 2) {
- CollectAllGarbage(
- kReduceMemoryFootprintMask | kAbortIncrementalMarkingMask,
- GarbageCollectionReason::kMemoryPressure,
- kGCCallbackFlagCollectAllAvailableGarbage);
+ CollectAllGarbage(kReduceMemoryFootprintMask,
+ GarbageCollectionReason::kMemoryPressure,
+ kGCCallbackFlagCollectAllAvailableGarbage);
} else {
if (FLAG_incremental_marking && incremental_marking()->IsStopped()) {
StartIncrementalMarking(kReduceMemoryFootprintMask,
@@ -3617,13 +3246,27 @@ void Heap::MemoryPressureNotification(MemoryPressureLevel level,
} else {
ExecutionAccess access(isolate());
isolate()->stack_guard()->RequestGC();
- V8::GetCurrentPlatform()->CallOnForegroundThread(
- reinterpret_cast<v8::Isolate*>(isolate()),
- new MemoryPressureInterruptTask(this));
+ auto taskrunner = V8::GetCurrentPlatform()->GetForegroundTaskRunner(
+ reinterpret_cast<v8::Isolate*>(isolate()));
+ taskrunner->PostTask(
+ base::make_unique<MemoryPressureInterruptTask>(this));
}
}
}
+void Heap::EagerlyFreeExternalMemory() {
+ for (Page* page : *old_space()) {
+ if (!page->SweepingDone()) {
+ base::LockGuard<base::Mutex> guard(page->mutex());
+ if (!page->SweepingDone()) {
+ ArrayBufferTracker::FreeDead(
+ page, mark_compact_collector()->non_atomic_marking_state());
+ }
+ }
+ }
+ memory_allocator()->unmapper()->EnsureUnmappingCompleted();
+}
+
void Heap::AddNearHeapLimitCallback(v8::NearHeapLimitCallback callback,
void* data) {
const size_t kMaxCallbacks = 100;
@@ -3833,16 +3476,15 @@ bool Heap::IsValidAllocationSpace(AllocationSpace space) {
}
}
-
-bool Heap::RootIsImmortalImmovable(int root_index) {
+bool Heap::RootIsImmortalImmovable(RootIndex root_index) {
switch (root_index) {
-#define IMMORTAL_IMMOVABLE_ROOT(name) case Heap::k##name##RootIndex:
+#define IMMORTAL_IMMOVABLE_ROOT(name) case RootIndex::k##name:
IMMORTAL_IMMOVABLE_ROOT_LIST(IMMORTAL_IMMOVABLE_ROOT)
#undef IMMORTAL_IMMOVABLE_ROOT
-#define INTERNALIZED_STRING(name, value) case Heap::k##name##RootIndex:
- INTERNALIZED_STRING_LIST(INTERNALIZED_STRING)
+#define INTERNALIZED_STRING(_, name, value) case RootIndex::k##name:
+ INTERNALIZED_STRING_LIST_GENERATOR(INTERNALIZED_STRING, /* not used */)
#undef INTERNALIZED_STRING
-#define STRING_TYPE(NAME, size, name, Name) case Heap::k##Name##MapRootIndex:
+#define STRING_TYPE(NAME, size, name, Name) case RootIndex::k##Name##Map:
STRING_TYPE_LIST(STRING_TYPE)
#undef STRING_TYPE
return true;
@@ -3867,7 +3509,7 @@ class VerifyReadOnlyPointersVisitor : public VerifyPointersVisitor {
for (MaybeObject** current = start; current < end; current++) {
HeapObject* object;
- if ((*current)->ToStrongOrWeakHeapObject(&object)) {
+ if ((*current)->GetHeapObject(&object)) {
CHECK(heap_->InReadOnlySpace(object));
}
}
@@ -3963,10 +3605,9 @@ class OldToNewSlotVerifyingVisitor : public SlotVerifyingVisitor {
: SlotVerifyingVisitor(untyped, typed) {}
bool ShouldHaveBeenRecorded(HeapObject* host, MaybeObject* target) override {
- DCHECK_IMPLIES(
- target->IsStrongOrWeakHeapObject() && Heap::InNewSpace(target),
- Heap::InToSpace(target));
- return target->IsStrongOrWeakHeapObject() && Heap::InNewSpace(target) &&
+ DCHECK_IMPLIES(target->IsStrongOrWeak() && Heap::InNewSpace(target),
+ Heap::InToSpace(target));
+ return target->IsStrongOrWeak() && Heap::InNewSpace(target) &&
!Heap::InNewSpace(host);
}
};
@@ -4077,9 +3718,8 @@ void Heap::IterateWeakRoots(RootVisitor* v, VisitMode mode) {
const bool isMinorGC = mode == VISIT_ALL_IN_SCAVENGE ||
mode == VISIT_ALL_IN_MINOR_MC_MARK ||
mode == VISIT_ALL_IN_MINOR_MC_UPDATE;
- v->VisitRootPointer(
- Root::kStringTable, nullptr,
- reinterpret_cast<Object**>(&roots_[kStringTableRootIndex]));
+ v->VisitRootPointer(Root::kStringTable, nullptr,
+ &roots_[RootIndex::kStringTable]);
v->Synchronize(VisitorSynchronization::kStringTable);
if (!isMinorGC && mode != VISIT_ALL_IN_SWEEP_NEWSPACE &&
mode != VISIT_FOR_SERIALIZATION) {
@@ -4094,8 +3734,8 @@ void Heap::IterateWeakRoots(RootVisitor* v, VisitMode mode) {
void Heap::IterateSmiRoots(RootVisitor* v) {
// Acquire execution access since we are going to read stack limit values.
ExecutionAccess access(isolate());
- v->VisitRootPointers(Root::kSmiRootList, nullptr, &roots_[kSmiRootsStart],
- &roots_[kRootListLength]);
+ v->VisitRootPointers(Root::kSmiRootList, nullptr, roots_.smi_roots_begin(),
+ roots_.smi_roots_end());
v->Synchronize(VisitorSynchronization::kSmiRootList);
}
@@ -4152,8 +3792,13 @@ void Heap::IterateStrongRoots(RootVisitor* v, VisitMode mode) {
const bool isMinorGC = mode == VISIT_ALL_IN_SCAVENGE ||
mode == VISIT_ALL_IN_MINOR_MC_MARK ||
mode == VISIT_ALL_IN_MINOR_MC_UPDATE;
- v->VisitRootPointers(Root::kStrongRootList, nullptr, &roots_[0],
- &roots_[kStrongRootListLength]);
+ // Garbage collection can skip over the read-only roots.
+ const bool isGC = mode != VISIT_ALL && mode != VISIT_FOR_SERIALIZATION &&
+ mode != VISIT_ONLY_STRONG_FOR_SERIALIZATION;
+ Object** start =
+ isGC ? roots_.read_only_roots_end() : roots_.strong_roots_begin();
+ v->VisitRootPointers(Root::kStrongRootList, nullptr, start,
+ roots_.strong_roots_end());
v->Synchronize(VisitorSynchronization::kStrongRootList);
isolate_->bootstrapper()->Iterate(v);
@@ -4192,17 +3837,19 @@ void Heap::IterateStrongRoots(RootVisitor* v, VisitMode mode) {
// global handles need to be added manually.
break;
case VISIT_ONLY_STRONG:
+ case VISIT_ONLY_STRONG_FOR_SERIALIZATION:
isolate_->global_handles()->IterateStrongRoots(v);
break;
case VISIT_ALL_IN_SCAVENGE:
isolate_->global_handles()->IterateNewSpaceStrongAndDependentRoots(v);
break;
case VISIT_ALL_IN_MINOR_MC_MARK:
- // Global handles are processed manually be the minor MC.
+ // Global handles are processed manually by the minor MC.
break;
case VISIT_ALL_IN_MINOR_MC_UPDATE:
- // Global handles are processed manually be the minor MC.
+ // Global handles are processed manually by the minor MC.
break;
+ case VISIT_ALL_BUT_READ_ONLY:
case VISIT_ALL_IN_SWEEP_NEWSPACE:
case VISIT_ALL:
isolate_->global_handles()->IterateAllRoots(v);
@@ -4706,6 +4353,9 @@ void Heap::SetUp() {
heap_controller_ = new HeapController(this);
mark_compact_collector_ = new MarkCompactCollector(this);
+
+ scavenger_collector_ = new ScavengerCollector(this);
+
incremental_marking_ =
new IncrementalMarking(this, mark_compact_collector_->marking_worklist(),
mark_compact_collector_->weak_objects());
@@ -4715,10 +4365,11 @@ void Heap::SetUp() {
mark_compact_collector_->marking_worklist();
concurrent_marking_ = new ConcurrentMarking(
this, marking_worklist->shared(), marking_worklist->bailout(),
- marking_worklist->on_hold(), mark_compact_collector_->weak_objects());
+ marking_worklist->on_hold(), mark_compact_collector_->weak_objects(),
+ marking_worklist->embedder());
} else {
- concurrent_marking_ =
- new ConcurrentMarking(this, nullptr, nullptr, nullptr, nullptr);
+ concurrent_marking_ = new ConcurrentMarking(this, nullptr, nullptr, nullptr,
+ nullptr, nullptr);
}
for (int i = FIRST_SPACE; i <= LAST_SPACE; i++) {
@@ -4727,7 +4378,8 @@ void Heap::SetUp() {
space_[RO_SPACE] = read_only_space_ = new ReadOnlySpace(this);
space_[NEW_SPACE] = new_space_ =
- new NewSpace(this, initial_semispace_size_, max_semi_space_size_);
+ new NewSpace(this, memory_allocator_->data_page_allocator(),
+ initial_semispace_size_, max_semi_space_size_);
space_[OLD_SPACE] = old_space_ = new OldSpace(this);
space_[CODE_SPACE] = code_space_ = new CodeSpace(this);
space_[MAP_SPACE] = map_space_ = new MapSpace(this);
@@ -4809,15 +4461,15 @@ void Heap::SetStackLimits() {
// Set up the special root array entries containing the stack limits.
// These are actually addresses, but the tag makes the GC ignore it.
- roots_[kStackLimitRootIndex] = reinterpret_cast<Object*>(
+ roots_[RootIndex::kStackLimit] = reinterpret_cast<Object*>(
(isolate_->stack_guard()->jslimit() & ~kSmiTagMask) | kSmiTag);
- roots_[kRealStackLimitRootIndex] = reinterpret_cast<Object*>(
+ roots_[RootIndex::kRealStackLimit] = reinterpret_cast<Object*>(
(isolate_->stack_guard()->real_jslimit() & ~kSmiTagMask) | kSmiTag);
}
void Heap::ClearStackLimits() {
- roots_[kStackLimitRootIndex] = Smi::kZero;
- roots_[kRealStackLimitRootIndex] = Smi::kZero;
+ roots_[RootIndex::kStackLimit] = Smi::kZero;
+ roots_[RootIndex::kRealStackLimit] = Smi::kZero;
}
int Heap::NextAllocationTimeout(int current_timeout) {
@@ -4873,6 +4525,10 @@ void Heap::SetEmbedderHeapTracer(EmbedderHeapTracer* tracer) {
local_embedder_heap_tracer()->SetRemoteTracer(tracer);
}
+EmbedderHeapTracer* Heap::GetEmbedderHeapTracer() const {
+ return local_embedder_heap_tracer()->remote_tracer();
+}
+
void Heap::TracePossibleWrapper(JSObject* js_object) {
DCHECK(js_object->IsApiWrapper());
if (js_object->GetEmbedderFieldCount() >= 2 &&
@@ -4961,6 +4617,11 @@ void Heap::TearDown() {
}
#endif // ENABLE_MINOR_MC
+ if (scavenger_collector_ != nullptr) {
+ delete scavenger_collector_;
+ scavenger_collector_ = nullptr;
+ }
+
if (array_buffer_collector_ != nullptr) {
delete array_buffer_collector_;
array_buffer_collector_ = nullptr;
@@ -5100,7 +4761,7 @@ Handle<WeakArrayList> CompactWeakArrayList(Heap* heap,
int copy_to = 0;
for (int i = 0; i < array->length(); i++) {
MaybeObject* element = array->Get(i);
- if (element->IsClearedWeakHeapObject()) continue;
+ if (element->IsCleared()) continue;
new_array->Set(copy_to++, element);
}
new_array->set_length(copy_to);
@@ -5174,11 +4835,11 @@ void Heap::CompactRetainedMaps(WeakArrayList* retained_maps) {
// This loop compacts the array by removing cleared weak cells.
for (int i = 0; i < length; i += 2) {
MaybeObject* maybe_object = retained_maps->Get(i);
- if (maybe_object->IsClearedWeakHeapObject()) {
+ if (maybe_object->IsCleared()) {
continue;
}
- DCHECK(maybe_object->IsWeakHeapObject());
+ DCHECK(maybe_object->IsWeak());
MaybeObject* age = retained_maps->Get(i + 1);
DCHECK(age->IsSmi());
@@ -5268,17 +4929,19 @@ void Heap::ClearRecordedSlot(HeapObject* object, Object** slot) {
}
}
-bool Heap::HasRecordedSlot(HeapObject* object, Object** slot) {
- if (InNewSpace(object)) {
- return false;
- }
+#ifdef DEBUG
+void Heap::VerifyClearedSlot(HeapObject* object, Object** slot) {
+ if (InNewSpace(object)) return;
Address slot_addr = reinterpret_cast<Address>(slot);
Page* page = Page::FromAddress(slot_addr);
DCHECK_EQ(page->owner()->identity(), OLD_SPACE);
store_buffer()->MoveAllEntriesToRememberedSet();
- return RememberedSet<OLD_TO_NEW>::Contains(page, slot_addr) ||
- RememberedSet<OLD_TO_OLD>::Contains(page, slot_addr);
+ CHECK(!RememberedSet<OLD_TO_NEW>::Contains(page, slot_addr));
+ // Old to old slots are filtered with invalidated slots.
+ CHECK_IMPLIES(RememberedSet<OLD_TO_OLD>::Contains(page, slot_addr),
+ page->RegisteredObjectWithInvalidatedSlots(object));
}
+#endif
void Heap::ClearRecordedSlotRange(Address start, Address end) {
Page* page = Page::FromAddress(start);
@@ -5308,9 +4971,7 @@ PagedSpace* PagedSpaces::next() {
SpaceIterator::SpaceIterator(Heap* heap)
: heap_(heap), current_space_(FIRST_SPACE - 1) {}
-SpaceIterator::~SpaceIterator() {
-}
-
+SpaceIterator::~SpaceIterator() = default;
bool SpaceIterator::has_next() {
// Iterate until no more spaces.
@@ -5325,7 +4986,7 @@ Space* SpaceIterator::next() {
class HeapObjectsFilter {
public:
- virtual ~HeapObjectsFilter() {}
+ virtual ~HeapObjectsFilter() = default;
virtual bool SkipObject(HeapObject* object) = 0;
};
@@ -5336,14 +4997,14 @@ class UnreachableObjectsFilter : public HeapObjectsFilter {
MarkReachableObjects();
}
- ~UnreachableObjectsFilter() {
+ ~UnreachableObjectsFilter() override {
for (auto it : reachable_) {
delete it.second;
it.second = nullptr;
}
}
- bool SkipObject(HeapObject* object) {
+ bool SkipObject(HeapObject* object) override {
if (object->IsFiller()) return true;
MemoryChunk* chunk = MemoryChunk::FromAddress(object->address());
if (reachable_.count(chunk) == 0) return true;
@@ -5396,7 +5057,7 @@ class UnreachableObjectsFilter : public HeapObjectsFilter {
// Treat weak references as strong.
for (MaybeObject** p = start; p < end; p++) {
HeapObject* heap_object;
- if ((*p)->ToStrongOrWeakHeapObject(&heap_object)) {
+ if ((*p)->GetHeapObject(&heap_object)) {
if (filter_->MarkAsReachable(heap_object)) {
marking_stack_.push_back(heap_object);
}
@@ -5411,7 +5072,7 @@ class UnreachableObjectsFilter : public HeapObjectsFilter {
void MarkReachableObjects() {
MarkingVisitor visitor(this);
- heap_->IterateRoots(&visitor, VISIT_ALL);
+ heap_->IterateRoots(&visitor, VISIT_ALL_BUT_READ_ONLY);
visitor.TransitiveClosure();
}
@@ -5597,24 +5258,6 @@ void Heap::UnregisterStrongRoots(Object** start) {
}
}
-bool Heap::IsDeserializeLazyHandler(Code* code) {
- return (code == deserialize_lazy_handler() ||
- code == deserialize_lazy_handler_wide() ||
- code == deserialize_lazy_handler_extra_wide());
-}
-
-void Heap::SetDeserializeLazyHandler(Code* code) {
- set_deserialize_lazy_handler(code);
-}
-
-void Heap::SetDeserializeLazyHandlerWide(Code* code) {
- set_deserialize_lazy_handler_wide(code);
-}
-
-void Heap::SetDeserializeLazyHandlerExtraWide(Code* code) {
- set_deserialize_lazy_handler_extra_wide(code);
-}
-
void Heap::SetBuiltinsConstantsTable(FixedArray* cache) {
set_builtins_constants_table(cache);
}
@@ -5723,11 +5366,11 @@ void VerifyPointersVisitor::VerifyPointers(HeapObject* host,
MaybeObject** end) {
for (MaybeObject** current = start; current < end; current++) {
HeapObject* object;
- if ((*current)->ToStrongOrWeakHeapObject(&object)) {
+ if ((*current)->GetHeapObject(&object)) {
CHECK(heap_->Contains(object));
CHECK(object->map()->IsMap());
} else {
- CHECK((*current)->IsSmi() || (*current)->IsClearedWeakHeapObject());
+ CHECK((*current)->IsSmi() || (*current)->IsCleared());
}
}
}
diff --git a/deps/v8/src/heap/heap.h b/deps/v8/src/heap/heap.h
index 2e750d56fa..c99f0d424e 100644
--- a/deps/v8/src/heap/heap.h
+++ b/deps/v8/src/heap/heap.h
@@ -13,6 +13,7 @@
// Clients of this interface shouldn't depend on lots of heap internals.
// Do not include anything from src/heap here!
+#include "include/v8-internal.h"
#include "include/v8.h"
#include "src/accessors.h"
#include "src/allocation.h"
@@ -40,6 +41,7 @@ class HeapTester;
class TestMemoryAllocatorScope;
} // namespace heap
+class AllocationMemento;
class ObjectBoilerplateDescription;
class BytecodeArray;
class CodeDataContainer;
@@ -50,6 +52,10 @@ class JSArrayBuffer;
class ExternalString;
using v8::MemoryPressureLevel;
+// Adapts PRIVATE_SYMBOL_LIST_GERNATOR entry to IMMORTAL_IMMOVABLE_ROOT_LIST
+// entry
+#define PRIVATE_SYMBOL_LIST_TO_IMMORTAL_IMMOVABLE_LIST_ADAPTER(V, name) V(name)
+
// Heap roots that are known to be immortal immovable, for which we can safely
// skip write barriers. This list is not complete and has omissions.
#define IMMORTAL_IMMOVABLE_ROOT_LIST(V) \
@@ -57,6 +63,7 @@ using v8::MemoryPressureLevel;
V(ArgumentsMarkerMap) \
V(ArrayBufferNeuteringProtector) \
V(ArrayIteratorProtector) \
+ V(AwaitContextMap) \
V(BigIntMap) \
V(BlockContextMap) \
V(ObjectBoilerplateDescriptionMap) \
@@ -143,6 +150,7 @@ using v8::MemoryPressureLevel;
V(TypedArraySpeciesProtector) \
V(PromiseSpeciesProtector) \
V(StaleRegister) \
+ V(StringIteratorProtector) \
V(StringLengthProtector) \
V(StringTableMap) \
V(SymbolMap) \
@@ -162,7 +170,8 @@ using v8::MemoryPressureLevel;
V(WeakArrayListMap) \
V(WithContextMap) \
V(empty_string) \
- PRIVATE_SYMBOL_LIST(V)
+ PRIVATE_SYMBOL_LIST_GENERATOR( \
+ PRIVATE_SYMBOL_LIST_TO_IMMORTAL_IMMOVABLE_LIST_ADAPTER, V)
class AllocationObserver;
class ArrayBufferCollector;
@@ -189,6 +198,7 @@ class PagedSpace;
class RootVisitor;
class ScavengeJob;
class Scavenger;
+class ScavengerCollector;
class Space;
class StoreBuffer;
class StressScavengeObserver;
@@ -206,6 +216,8 @@ enum class ClearRecordedSlots { kYes, kNo };
enum class ClearFreedMemoryMode { kClearFreedMemory, kDontClearFreedMemory };
+enum ExternalBackingStoreType { kArrayBuffer, kExternalString, kNumTypes };
+
enum class FixedArrayVisitationMode { kRegular, kIncremental };
enum class TraceRetainingPathMode { kEnabled, kDisabled };
@@ -305,54 +317,6 @@ struct CommentStatistic {
class Heap {
public:
- // Declare all the root indices. This defines the root list order.
- // clang-format off
- enum RootListIndex {
-#define DECL(type, name, camel_name) k##camel_name##RootIndex,
- STRONG_ROOT_LIST(DECL)
-#undef DECL
-
-#define DECL(name, str) k##name##RootIndex,
- INTERNALIZED_STRING_LIST(DECL)
-#undef DECL
-
-#define DECL(name) k##name##RootIndex,
- PRIVATE_SYMBOL_LIST(DECL)
-#undef DECL
-
-#define DECL(name, description) k##name##RootIndex,
- PUBLIC_SYMBOL_LIST(DECL)
- WELL_KNOWN_SYMBOL_LIST(DECL)
-#undef DECL
-
-#define DECL(accessor_name, AccessorName) k##AccessorName##AccessorRootIndex,
- ACCESSOR_INFO_LIST(DECL)
-#undef DECL
-
-#define DECL(NAME, Name, name) k##Name##MapRootIndex,
- STRUCT_LIST(DECL)
-#undef DECL
-
-#define DECL(NAME, Name, Size, name) k##Name##Size##MapRootIndex,
- ALLOCATION_SITE_LIST(DECL)
-#undef DECL
-
-#define DECL(NAME, Name, Size, name) k##Name##Size##MapRootIndex,
- DATA_HANDLER_LIST(DECL)
-#undef DECL
-
- kStringTableRootIndex,
-
-#define DECL(type, name, camel_name) k##camel_name##RootIndex,
- SMI_ROOT_LIST(DECL)
-#undef DECL
-
- kRootListLength,
- kStrongRootListLength = kStringTableRootIndex,
- kSmiRootsStart = kStringTableRootIndex + 1
- };
- // clang-format on
-
enum FindMementoMode { kForRuntime, kForGC };
enum HeapState {
@@ -399,11 +363,6 @@ class Heap {
static const int kNoGCFlags = 0;
static const int kReduceMemoryFootprintMask = 1;
- static const int kAbortIncrementalMarkingMask = 2;
- static const int kFinalizeIncrementalMarkingMask = 4;
-
- // Making the heap iterable requires us to abort incremental marking.
- static const int kMakeHeapIterableMask = kAbortIncrementalMarkingMask;
// The roots that have an index less than this are always in old space.
static const int kOldSpaceRoots = 0x20;
@@ -413,13 +372,18 @@ class Heap {
static const int kMinPromotedPercentForFastPromotionMode = 90;
- STATIC_ASSERT(kUndefinedValueRootIndex ==
+ STATIC_ASSERT(static_cast<int>(RootIndex::kUndefinedValue) ==
Internals::kUndefinedValueRootIndex);
- STATIC_ASSERT(kTheHoleValueRootIndex == Internals::kTheHoleValueRootIndex);
- STATIC_ASSERT(kNullValueRootIndex == Internals::kNullValueRootIndex);
- STATIC_ASSERT(kTrueValueRootIndex == Internals::kTrueValueRootIndex);
- STATIC_ASSERT(kFalseValueRootIndex == Internals::kFalseValueRootIndex);
- STATIC_ASSERT(kempty_stringRootIndex == Internals::kEmptyStringRootIndex);
+ STATIC_ASSERT(static_cast<int>(RootIndex::kTheHoleValue) ==
+ Internals::kTheHoleValueRootIndex);
+ STATIC_ASSERT(static_cast<int>(RootIndex::kNullValue) ==
+ Internals::kNullValueRootIndex);
+ STATIC_ASSERT(static_cast<int>(RootIndex::kTrueValue) ==
+ Internals::kTrueValueRootIndex);
+ STATIC_ASSERT(static_cast<int>(RootIndex::kFalseValue) ==
+ Internals::kFalseValueRootIndex);
+ STATIC_ASSERT(static_cast<int>(RootIndex::kempty_string) ==
+ Internals::kEmptyStringRootIndex);
// Calculates the maximum amount of filler that could be required by the
// given alignment.
@@ -430,14 +394,14 @@ class Heap {
void FatalProcessOutOfMemory(const char* location);
- V8_EXPORT_PRIVATE static bool RootIsImmortalImmovable(int root_index);
+ V8_EXPORT_PRIVATE static bool RootIsImmortalImmovable(RootIndex root_index);
// Checks whether the space is valid.
static bool IsValidAllocationSpace(AllocationSpace space);
// Generated code can embed direct references to non-writable roots if
// they are in new space.
- static bool RootCanBeWrittenAfterInitialization(RootListIndex root_index);
+ static bool RootCanBeWrittenAfterInitialization(RootIndex root_index);
// Zapping is needed for verify heap, and always done in debug builds.
static inline bool ShouldZapGarbage() {
@@ -566,8 +530,8 @@ class Heap {
// Traverse all the allocaions_sites [nested_site and weak_next] in the list
// and foreach call the visitor
- void ForeachAllocationSite(Object* list,
- std::function<void(AllocationSite*)> visitor);
+ void ForeachAllocationSite(
+ Object* list, const std::function<void(AllocationSite*)>& visitor);
// Number of mark-sweeps.
int ms_count() const { return ms_count_; }
@@ -688,8 +652,7 @@ class Heap {
external_memory_concurrently_freed_ = 0;
}
- void ProcessMovedExternalString(Page* old_page, Page* new_page,
- ExternalString* string);
+ size_t backing_store_bytes() const { return backing_store_bytes_; }
void CompactWeakArrayLists(PretenureFlag pretenure);
@@ -811,36 +774,29 @@ class Heap {
friend class ReadOnlyRoots;
public:
+ RootsTable& roots_table() { return roots_; }
+
// Heap root getters.
-#define ROOT_ACCESSOR(type, name, camel_name) inline type* name();
+#define ROOT_ACCESSOR(type, name, CamelName) inline type* name();
MUTABLE_ROOT_LIST(ROOT_ACCESSOR)
#undef ROOT_ACCESSOR
-#define DATA_HANDLER_MAP_ACCESSOR(NAME, Name, Size, name) \
- inline Map* name##_map();
- DATA_HANDLER_LIST(DATA_HANDLER_MAP_ACCESSOR)
-#undef DATA_HANDLER_MAP_ACCESSOR
-
-#define ACCESSOR_INFO_ACCESSOR(accessor_name, AccessorName) \
- inline AccessorInfo* accessor_name##_accessor();
- ACCESSOR_INFO_LIST(ACCESSOR_INFO_ACCESSOR)
-#undef ACCESSOR_INFO_ACCESSOR
-
- Object* root(RootListIndex index) { return roots_[index]; }
- Handle<Object> root_handle(RootListIndex index) {
+ Object* root(RootIndex index) { return roots_[index]; }
+ Handle<Object> root_handle(RootIndex index) {
return Handle<Object>(&roots_[index]);
}
+
+ bool IsRootHandleLocation(Object** handle_location, RootIndex* index) const {
+ return roots_.IsRootHandleLocation(handle_location, index);
+ }
+
template <typename T>
- bool IsRootHandle(Handle<T> handle, RootListIndex* index) const {
- Object** const handle_location = bit_cast<Object**>(handle.address());
- if (handle_location >= &roots_[kRootListLength]) return false;
- if (handle_location < &roots_[0]) return false;
- *index = static_cast<RootListIndex>(handle_location - &roots_[0]);
- return true;
+ bool IsRootHandle(Handle<T> handle, RootIndex* index) const {
+ return roots_.IsRootHandle(handle, index);
}
// Generated code can embed this address to get access to the roots.
- Object** roots_array_start() { return roots_; }
+ Object** roots_array_start() { return roots_.roots_; }
ExternalReferenceTable* external_reference_table() {
DCHECK(external_reference_table_.is_initialized());
@@ -868,23 +824,23 @@ class Heap {
void SetRootCodeStubs(SimpleNumberDictionary* value);
void SetRootMaterializedObjects(FixedArray* objects) {
- roots_[kMaterializedObjectsRootIndex] = objects;
+ roots_[RootIndex::kMaterializedObjects] = objects;
}
void SetRootScriptList(Object* value) {
- roots_[kScriptListRootIndex] = value;
+ roots_[RootIndex::kScriptList] = value;
}
void SetRootStringTable(StringTable* value) {
- roots_[kStringTableRootIndex] = value;
+ roots_[RootIndex::kStringTable] = value;
}
void SetRootNoScriptSharedFunctionInfos(Object* value) {
- roots_[kNoScriptSharedFunctionInfosRootIndex] = value;
+ roots_[RootIndex::kNoScriptSharedFunctionInfos] = value;
}
void SetMessageListeners(TemplateList* value) {
- roots_[kMessageListenersRootIndex] = value;
+ roots_[RootIndex::kMessageListeners] = value;
}
// Set the stack limit in the roots_ array. Some architectures generate
@@ -897,20 +853,11 @@ class Heap {
void ClearStackLimits();
// Generated code can treat direct references to this root as constant.
- bool RootCanBeTreatedAsConstant(RootListIndex root_index);
-
- Map* MapForFixedTypedArray(ExternalArrayType array_type);
- Map* MapForFixedTypedArray(ElementsKind elements_kind);
- FixedTypedArrayBase* EmptyFixedTypedArrayForMap(const Map* map);
+ bool RootCanBeTreatedAsConstant(RootIndex root_index);
void RegisterStrongRoots(Object** start, Object** end);
void UnregisterStrongRoots(Object** start);
- bool IsDeserializeLazyHandler(Code* code);
- void SetDeserializeLazyHandler(Code* code);
- void SetDeserializeLazyHandlerWide(Code* code);
- void SetDeserializeLazyHandlerExtraWide(Code* code);
-
void SetBuiltinsConstantsTable(FixedArray* cache);
// ===========================================================================
@@ -935,9 +882,7 @@ class Heap {
AllocationSpace space, GarbageCollectionReason gc_reason,
const GCCallbackFlags gc_callback_flags = kNoGCCallbackFlags);
- // Performs a full garbage collection. If (flags & kMakeHeapIterableMask) is
- // non-zero, then the slower precise sweeper is used, which leaves the heap
- // in a state where we can iterate over the heap visiting all objects.
+ // Performs a full garbage collection.
V8_EXPORT_PRIVATE void CollectAllGarbage(
int flags, GarbageCollectionReason gc_reason,
const GCCallbackFlags gc_callback_flags = kNoGCCallbackFlags);
@@ -945,6 +890,13 @@ class Heap {
// Last hope GC, should try to squeeze as much as possible.
void CollectAllAvailableGarbage(GarbageCollectionReason gc_reason);
+ // Precise garbage collection that potentially finalizes already running
+ // incremental marking before performing an atomic garbage collection.
+ // Only use if absolutely necessary or in tests to avoid floating garbage!
+ void PreciseCollectAllGarbage(
+ int flags, GarbageCollectionReason gc_reason,
+ const GCCallbackFlags gc_callback_flags = kNoGCCallbackFlags);
+
// Reports and external memory pressure event, either performs a major GC or
// completes incremental marking in order to free external resources.
void ReportExternalMemoryPressure();
@@ -1002,7 +954,9 @@ class Heap {
void ClearRecordedSlot(HeapObject* object, Object** slot);
void ClearRecordedSlotRange(Address start, Address end);
- bool HasRecordedSlot(HeapObject* object, Object** slot);
+#ifdef DEBUG
+ void VerifyClearedSlot(HeapObject* object, Object** slot);
+#endif
// ===========================================================================
// Incremental marking API. ==================================================
@@ -1084,10 +1038,13 @@ class Heap {
// Embedder heap tracer support. =============================================
// ===========================================================================
- LocalEmbedderHeapTracer* local_embedder_heap_tracer() {
+ LocalEmbedderHeapTracer* local_embedder_heap_tracer() const {
return local_embedder_heap_tracer_;
}
+
void SetEmbedderHeapTracer(EmbedderHeapTracer* tracer);
+ EmbedderHeapTracer* GetEmbedderHeapTracer() const;
+
void TracePossibleWrapper(JSObject* js_object);
void RegisterExternallyReferencedObject(Object** object);
void SetEmbedderStackStateForNextFinalizaton(
@@ -1109,6 +1066,9 @@ class Heap {
// data and clearing the resource pointer.
inline void FinalizeExternalString(String* string);
+ static String* UpdateNewSpaceReferenceInExternalStringTableEntry(
+ Heap* heap, Object** pointer);
+
// ===========================================================================
// Methods checking/returning the space of a given object/address. ===========
// ===========================================================================
@@ -1538,18 +1498,18 @@ class Heap {
struct StringTypeTable {
InstanceType type;
int size;
- RootListIndex index;
+ RootIndex index;
};
struct ConstantStringTable {
const char* contents;
- RootListIndex index;
+ RootIndex index;
};
struct StructTable {
InstanceType type;
int size;
- RootListIndex index;
+ RootIndex index;
};
struct GCCallbackTuple {
@@ -1584,13 +1544,8 @@ class Heap {
static const int kInitialFeedbackCapacity = 256;
- static const int kMaxScavengerTasks = 8;
-
Heap();
- static String* UpdateNewSpaceReferenceInExternalStringTableEntry(
- Heap* heap, Object** pointer);
-
// Selects the proper allocation space based on the pretenuring decision.
static AllocationSpace SelectSpace(PretenureFlag pretenure) {
switch (pretenure) {
@@ -1609,7 +1564,7 @@ class Heap {
return 0;
}
-#define ROOT_ACCESSOR(type, name, camel_name) \
+#define ROOT_ACCESSOR(type, name, CamelName) \
inline void set_##name(type* value);
ROOT_LIST(ROOT_ACCESSOR)
#undef ROOT_ACCESSOR
@@ -1618,22 +1573,12 @@ class Heap {
void set_current_gc_flags(int flags) {
current_gc_flags_ = flags;
- DCHECK(!ShouldFinalizeIncrementalMarking() ||
- !ShouldAbortIncrementalMarking());
}
inline bool ShouldReduceMemory() const {
return (current_gc_flags_ & kReduceMemoryFootprintMask) != 0;
}
- inline bool ShouldAbortIncrementalMarking() const {
- return (current_gc_flags_ & kAbortIncrementalMarkingMask) != 0;
- }
-
- inline bool ShouldFinalizeIncrementalMarking() const {
- return (current_gc_flags_ & kFinalizeIncrementalMarkingMask) != 0;
- }
-
int NumberOfScavengeTasks();
// Checks whether a global GC is necessary
@@ -1733,6 +1678,8 @@ class Heap {
void CollectGarbageOnMemoryPressure();
+ void EagerlyFreeExternalMemory();
+
bool InvokeNearHeapLimitCallback();
void ComputeFastPromotionMode();
@@ -1841,6 +1788,12 @@ class Heap {
void CheckIneffectiveMarkCompact(size_t old_generation_size,
double mutator_utilization);
+ inline void IncrementExternalBackingStoreBytes(ExternalBackingStoreType type,
+ size_t amount);
+
+ inline void DecrementExternalBackingStoreBytes(ExternalBackingStoreType type,
+ size_t amount);
+
// ===========================================================================
// Growing strategy. =========================================================
// ===========================================================================
@@ -1962,28 +1915,28 @@ class Heap {
void PrintRetainingPath(HeapObject* object, RetainingPathOption option);
// The amount of external memory registered through the API.
- int64_t external_memory_;
+ int64_t external_memory_ = 0;
// The limit when to trigger memory pressure from the API.
- int64_t external_memory_limit_;
+ int64_t external_memory_limit_ = kExternalAllocationSoftLimit;
// Caches the amount of external memory registered at the last MC.
- int64_t external_memory_at_last_mark_compact_;
+ int64_t external_memory_at_last_mark_compact_ = 0;
// The amount of memory that has been freed concurrently.
- std::atomic<intptr_t> external_memory_concurrently_freed_;
+ std::atomic<intptr_t> external_memory_concurrently_freed_{0};
// This can be calculated directly from a pointer to the heap; however, it is
// more expedient to get at the isolate directly from within Heap methods.
- Isolate* isolate_;
+ Isolate* isolate_ = nullptr;
- Object* roots_[kRootListLength];
+ RootsTable roots_;
// This table is accessed from builtin code compiled into the snapshot, and
// thus its offset from roots_ must remain static. This is verified in
// Isolate::Init() using runtime checks.
static constexpr int kRootsExternalReferenceTableOffset =
- kRootListLength * kPointerSize;
+ static_cast<int>(RootIndex::kRootListLength) * kPointerSize;
ExternalReferenceTable external_reference_table_;
// As external references above, builtins are accessed through an offset from
@@ -2000,25 +1953,28 @@ class Heap {
static constexpr int kRootRegisterAddressableEndOffset =
kRootsBuiltinsOffset + Builtins::builtin_count * kPointerSize;
- size_t code_range_size_;
- size_t max_semi_space_size_;
- size_t initial_semispace_size_;
- size_t max_old_generation_size_;
+ size_t code_range_size_ = 0;
+ size_t max_semi_space_size_ = 8 * (kPointerSize / 4) * MB;
+ size_t initial_semispace_size_ = kMinSemiSpaceSizeInKB * KB;
+ size_t max_old_generation_size_ = 700ul * (kPointerSize / 4) * MB;
size_t initial_max_old_generation_size_;
size_t initial_old_generation_size_;
- bool old_generation_size_configured_;
- size_t maximum_committed_;
+ bool old_generation_size_configured_ = false;
+ size_t maximum_committed_ = 0;
+
+ // Backing store bytes (array buffers and external strings).
+ std::atomic<size_t> backing_store_bytes_{0};
// For keeping track of how much data has survived
// scavenge since last new space expansion.
- size_t survived_since_last_expansion_;
+ size_t survived_since_last_expansion_ = 0;
// ... and since the last scavenge.
- size_t survived_last_scavenge_;
+ size_t survived_last_scavenge_ = 0;
// This is not the depth of nested AlwaysAllocateScope's but rather a single
// count, as scopes can be acquired from multiple tasks (read: threads).
- std::atomic<size_t> always_allocate_scope_count_;
+ std::atomic<size_t> always_allocate_scope_count_{0};
// Stores the memory pressure level that set by MemoryPressureNotification
// and reset by a mark-compact garbage collection.
@@ -2028,74 +1984,75 @@ class Heap {
near_heap_limit_callbacks_;
// For keeping track of context disposals.
- int contexts_disposed_;
+ int contexts_disposed_ = 0;
// The length of the retained_maps array at the time of context disposal.
// This separates maps in the retained_maps array that were created before
// and after context disposal.
- int number_of_disposed_maps_;
-
- NewSpace* new_space_;
- OldSpace* old_space_;
- CodeSpace* code_space_;
- MapSpace* map_space_;
- LargeObjectSpace* lo_space_;
- NewLargeObjectSpace* new_lo_space_;
- ReadOnlySpace* read_only_space_;
+ int number_of_disposed_maps_ = 0;
+
+ NewSpace* new_space_ = nullptr;
+ OldSpace* old_space_ = nullptr;
+ CodeSpace* code_space_ = nullptr;
+ MapSpace* map_space_ = nullptr;
+ LargeObjectSpace* lo_space_ = nullptr;
+ NewLargeObjectSpace* new_lo_space_ = nullptr;
+ ReadOnlySpace* read_only_space_ = nullptr;
// Map from the space id to the space.
Space* space_[LAST_SPACE + 1];
// Determines whether code space is write-protected. This is essentially a
// race-free copy of the {FLAG_write_protect_code_memory} flag.
- bool write_protect_code_memory_;
+ bool write_protect_code_memory_ = false;
// Holds the number of open CodeSpaceMemoryModificationScopes.
- uintptr_t code_space_memory_modification_scope_depth_;
+ uintptr_t code_space_memory_modification_scope_depth_ = 0;
+
+ HeapState gc_state_ = NOT_IN_GC;
- HeapState gc_state_;
- int gc_post_processing_depth_;
+ int gc_post_processing_depth_ = 0;
// Returns the amount of external memory registered since last global gc.
uint64_t PromotedExternalMemorySize();
// How many "runtime allocations" happened.
- uint32_t allocations_count_;
+ uint32_t allocations_count_ = 0;
// Running hash over allocations performed.
- uint32_t raw_allocations_hash_;
+ uint32_t raw_allocations_hash_ = 0;
// Starts marking when stress_marking_percentage_% of the marking start limit
// is reached.
- int stress_marking_percentage_;
+ int stress_marking_percentage_ = 0;
// Observer that causes more frequent checks for reached incremental marking
// limit.
- AllocationObserver* stress_marking_observer_;
+ AllocationObserver* stress_marking_observer_ = nullptr;
// Observer that can cause early scavenge start.
- StressScavengeObserver* stress_scavenge_observer_;
+ StressScavengeObserver* stress_scavenge_observer_ = nullptr;
- bool allocation_step_in_progress_;
+ bool allocation_step_in_progress_ = false;
// The maximum percent of the marking limit reached wihout causing marking.
// This is tracked when specyfing --fuzzer-gc-analysis.
- double max_marking_limit_reached_;
+ double max_marking_limit_reached_ = 0.0;
// How many mark-sweep collections happened.
- unsigned int ms_count_;
+ unsigned int ms_count_ = 0;
// How many gc happened.
- unsigned int gc_count_;
+ unsigned int gc_count_ = 0;
// The number of Mark-Compact garbage collections that are considered as
// ineffective. See IsIneffectiveMarkCompact() predicate.
- int consecutive_ineffective_mark_compacts_;
+ int consecutive_ineffective_mark_compacts_ = 0;
static const uintptr_t kMmapRegionMask = 0xFFFFFFFFu;
- uintptr_t mmap_region_base_;
+ uintptr_t mmap_region_base_ = 0;
// For post mortem debugging.
- int remembered_unmapped_pages_index_;
+ int remembered_unmapped_pages_index_ = 0;
Address remembered_unmapped_pages_[kRememberedUnmappedPages];
// Limit that triggers a global GC on the next (normally caused) GC. This
@@ -2106,7 +2063,7 @@ class Heap {
// Indicates that inline bump-pointer allocation has been globally disabled
// for all spaces. This is used to disable allocations in generated code.
- bool inline_allocation_disabled_;
+ bool inline_allocation_disabled_ = false;
// Weak list heads, threaded through the objects.
// List heads are initialized lazily and contain the undefined_value at start.
@@ -2120,71 +2077,62 @@ class Heap {
int deferred_counters_[v8::Isolate::kUseCounterFeatureCount];
- GCTracer* tracer_;
-
- size_t promoted_objects_size_;
- double promotion_ratio_;
- double promotion_rate_;
- size_t semi_space_copied_object_size_;
- size_t previous_semi_space_copied_object_size_;
- double semi_space_copied_rate_;
- int nodes_died_in_new_space_;
- int nodes_copied_in_new_space_;
- int nodes_promoted_;
+ size_t promoted_objects_size_ = 0;
+ double promotion_ratio_ = 0.0;
+ double promotion_rate_ = 0.0;
+ size_t semi_space_copied_object_size_ = 0;
+ size_t previous_semi_space_copied_object_size_ = 0;
+ double semi_space_copied_rate_ = 0.0;
+ int nodes_died_in_new_space_ = 0;
+ int nodes_copied_in_new_space_ = 0;
+ int nodes_promoted_ = 0;
// This is the pretenuring trigger for allocation sites that are in maybe
// tenure state. When we switched to the maximum new space size we deoptimize
// the code that belongs to the allocation site and derive the lifetime
// of the allocation site.
- unsigned int maximum_size_scavenges_;
+ unsigned int maximum_size_scavenges_ = 0;
// Total time spent in GC.
double total_gc_time_ms_;
// Last time an idle notification happened.
- double last_idle_notification_time_;
+ double last_idle_notification_time_ = 0.0;
// Last time a garbage collection happened.
- double last_gc_time_;
-
- MarkCompactCollector* mark_compact_collector_;
- MinorMarkCompactCollector* minor_mark_compact_collector_;
-
- ArrayBufferCollector* array_buffer_collector_;
-
- MemoryAllocator* memory_allocator_;
-
- StoreBuffer* store_buffer_;
-
- HeapController* heap_controller_;
-
- IncrementalMarking* incremental_marking_;
- ConcurrentMarking* concurrent_marking_;
-
- GCIdleTimeHandler* gc_idle_time_handler_;
-
- MemoryReducer* memory_reducer_;
-
- ObjectStats* live_object_stats_;
- ObjectStats* dead_object_stats_;
-
- ScavengeJob* scavenge_job_;
- base::Semaphore parallel_scavenge_semaphore_;
-
- AllocationObserver* idle_scavenge_observer_;
+ double last_gc_time_ = 0.0;
+
+ GCTracer* tracer_ = nullptr;
+ MarkCompactCollector* mark_compact_collector_ = nullptr;
+ MinorMarkCompactCollector* minor_mark_compact_collector_ = nullptr;
+ ScavengerCollector* scavenger_collector_ = nullptr;
+ ArrayBufferCollector* array_buffer_collector_ = nullptr;
+ MemoryAllocator* memory_allocator_ = nullptr;
+ StoreBuffer* store_buffer_ = nullptr;
+ HeapController* heap_controller_ = nullptr;
+ IncrementalMarking* incremental_marking_ = nullptr;
+ ConcurrentMarking* concurrent_marking_ = nullptr;
+ GCIdleTimeHandler* gc_idle_time_handler_ = nullptr;
+ MemoryReducer* memory_reducer_ = nullptr;
+ ObjectStats* live_object_stats_ = nullptr;
+ ObjectStats* dead_object_stats_ = nullptr;
+ ScavengeJob* scavenge_job_ = nullptr;
+ AllocationObserver* idle_scavenge_observer_ = nullptr;
+ LocalEmbedderHeapTracer* local_embedder_heap_tracer_ = nullptr;
+ StrongRootsList* strong_roots_list_ = nullptr;
// This counter is increased before each GC and never reset.
// To account for the bytes allocated since the last GC, use the
// NewSpaceAllocationCounter() function.
- size_t new_space_allocation_counter_;
+ size_t new_space_allocation_counter_ = 0;
// This counter is increased before each GC and never reset. To
// account for the bytes allocated since the last GC, use the
// OldGenerationAllocationCounter() function.
- size_t old_generation_allocation_counter_at_last_gc_;
+ size_t old_generation_allocation_counter_at_last_gc_ = 0;
// The size of objects in old generation after the last MarkCompact GC.
- size_t old_generation_size_at_last_gc_;
+ size_t old_generation_size_at_last_gc_ = 0;
// The feedback storage is used to store allocation sites (keys) and how often
// they have been visited (values) by finding a memento behind an object. The
@@ -2196,20 +2144,20 @@ class Heap {
char trace_ring_buffer_[kTraceRingBufferSize];
// Used as boolean.
- uint8_t is_marking_flag_;
+ uint8_t is_marking_flag_ = 0;
// If it's not full then the data is from 0 to ring_buffer_end_. If it's
// full then the data is from ring_buffer_end_ to the end of the buffer and
// from 0 to ring_buffer_end_.
- bool ring_buffer_full_;
- size_t ring_buffer_end_;
+ bool ring_buffer_full_ = false;
+ size_t ring_buffer_end_ = 0;
// Flag is set when the heap has been configured. The heap can be repeatedly
// configured through the API until it is set up.
- bool configured_;
+ bool configured_ = false;
// Currently set GC flags that are respected by all GC components.
- int current_gc_flags_;
+ int current_gc_flags_ = Heap::kNoGCFlags;
// Currently set GC callback flags that are used to pass information between
// the embedder and V8's GC.
@@ -2219,34 +2167,30 @@ class Heap {
base::Mutex relocation_mutex_;
- int gc_callbacks_depth_;
-
- bool deserialization_complete_;
+ int gc_callbacks_depth_ = 0;
- StrongRootsList* strong_roots_list_;
+ bool deserialization_complete_ = false;
// The depth of HeapIterator nestings.
- int heap_iterator_depth_;
+ int heap_iterator_depth_ = 0;
- LocalEmbedderHeapTracer* local_embedder_heap_tracer_;
-
- bool fast_promotion_mode_;
+ bool fast_promotion_mode_ = false;
// Used for testing purposes.
- bool force_oom_;
- bool delay_sweeper_tasks_for_testing_;
+ bool force_oom_ = false;
+ bool delay_sweeper_tasks_for_testing_ = false;
- HeapObject* pending_layout_change_object_;
+ HeapObject* pending_layout_change_object_ = nullptr;
base::Mutex unprotected_memory_chunks_mutex_;
std::unordered_set<MemoryChunk*> unprotected_memory_chunks_;
- bool unprotected_memory_chunks_registry_enabled_;
+ bool unprotected_memory_chunks_registry_enabled_ = false;
#ifdef V8_ENABLE_ALLOCATION_TIMEOUT
// If the --gc-interval flag is set to a positive value, this
// variable holds the value indicating the number of allocations
// remain until the next failure and garbage collection.
- int allocation_timeout_;
+ int allocation_timeout_ = 0;
#endif // V8_ENABLE_ALLOCATION_TIMEOUT
std::map<HeapObject*, HeapObject*> retainer_;
@@ -2262,6 +2206,7 @@ class Heap {
// Classes in "heap" can be friends.
friend class AlwaysAllocateScope;
+ friend class ArrayBufferCollector;
friend class ConcurrentMarking;
friend class EphemeronHashTableMarkingTask;
friend class GCCallbacksScope;
@@ -2283,6 +2228,8 @@ class Heap {
friend class Page;
friend class PagedSpace;
friend class Scavenger;
+ friend class ScavengerCollector;
+ friend class Space;
friend class StoreBuffer;
friend class Sweeper;
friend class heap::TestMemoryAllocatorScope;
@@ -2297,7 +2244,8 @@ class Heap {
friend class heap::HeapTester;
FRIEND_TEST(HeapControllerTest, OldGenerationAllocationLimit);
-
+ FRIEND_TEST(HeapTest, ExternalLimitDefault);
+ FRIEND_TEST(HeapTest, ExternalLimitStaysAboveDefaultForExplicitHandling);
DISALLOW_COPY_AND_ASSIGN(Heap);
};
@@ -2417,7 +2365,7 @@ class VerifySmisVisitor : public RootVisitor {
// Space iterator for iterating over all the paged spaces of the heap: Map
// space, old space, code space and optionally read only space. Returns each
// space in turn, and null when it is done.
-class V8_EXPORT_PRIVATE PagedSpaces BASE_EMBEDDED {
+class V8_EXPORT_PRIVATE PagedSpaces {
public:
enum class SpacesSpecifier { kSweepablePagedSpaces, kAllPagedSpaces };
@@ -2460,7 +2408,7 @@ class SpaceIterator : public Malloced {
// nodes filtering uses GC marks, it can't be used during MS/MC GC
// phases. Also, it is forbidden to interrupt iteration in this mode,
// as this will leave heap objects marked (and thus, unusable).
-class HeapIterator BASE_EMBEDDED {
+class HeapIterator {
public:
enum HeapObjectsFiltering { kNoFiltering, kFilterUnreachable };
@@ -2487,7 +2435,7 @@ class HeapIterator BASE_EMBEDDED {
// Abstract base class for checking whether a weak object should be retained.
class WeakObjectRetainer {
public:
- virtual ~WeakObjectRetainer() {}
+ virtual ~WeakObjectRetainer() = default;
// Return whether this object should be retained. If nullptr is returned the
// object has no references. Otherwise the address of the retained object
@@ -2503,7 +2451,7 @@ class AllocationObserver {
: step_size_(step_size), bytes_to_next_step_(step_size) {
DCHECK_LE(kPointerSize, step_size);
}
- virtual ~AllocationObserver() {}
+ virtual ~AllocationObserver() = default;
// Called each time the observed space does an allocation step. This may be
// more frequently than the step_size we are monitoring (e.g. when there are
diff --git a/deps/v8/src/heap/incremental-marking-inl.h b/deps/v8/src/heap/incremental-marking-inl.h
index 19d6b22e4d..e19d62f4d4 100644
--- a/deps/v8/src/heap/incremental-marking-inl.h
+++ b/deps/v8/src/heap/incremental-marking-inl.h
@@ -6,6 +6,8 @@
#define V8_HEAP_INCREMENTAL_MARKING_INL_H_
#include "src/heap/incremental-marking.h"
+
+#include "src/heap/mark-compact-inl.h"
#include "src/isolate.h"
#include "src/objects-inl.h"
#include "src/objects/maybe-object.h"
@@ -13,6 +15,23 @@
namespace v8 {
namespace internal {
+void IncrementalMarking::TransferColor(HeapObject* from, HeapObject* to) {
+ if (atomic_marking_state()->IsBlack(to)) {
+ DCHECK(black_allocation());
+ return;
+ }
+
+ DCHECK(atomic_marking_state()->IsWhite(to));
+ if (atomic_marking_state()->IsGrey(from)) {
+ bool success = atomic_marking_state()->WhiteToGrey(to);
+ DCHECK(success);
+ USE(success);
+ } else if (atomic_marking_state()->IsBlack(from)) {
+ bool success = atomic_marking_state()->WhiteToBlack(to);
+ DCHECK(success);
+ USE(success);
+ }
+}
void IncrementalMarking::RecordWrite(HeapObject* obj, Object** slot,
Object* value) {
@@ -30,7 +49,7 @@ void IncrementalMarking::RecordMaybeWeakWrite(HeapObject* obj,
// When writing a weak reference, treat it as strong for the purposes of the
// marking barrier.
HeapObject* heap_object;
- if (IsMarking() && value->ToStrongOrWeakHeapObject(&heap_object)) {
+ if (IsMarking() && value->GetHeapObject(&heap_object)) {
RecordWriteSlow(obj, reinterpret_cast<HeapObjectReference**>(slot),
heap_object);
}
diff --git a/deps/v8/src/heap/incremental-marking-job.cc b/deps/v8/src/heap/incremental-marking-job.cc
index 7583aaaadf..96eff0508e 100644
--- a/deps/v8/src/heap/incremental-marking-job.cc
+++ b/deps/v8/src/heap/incremental-marking-job.cc
@@ -24,8 +24,9 @@ void IncrementalMarkingJob::ScheduleTask(Heap* heap) {
if (!task_pending_ && !heap->IsTearingDown()) {
v8::Isolate* isolate = reinterpret_cast<v8::Isolate*>(heap->isolate());
task_pending_ = true;
- auto task = new Task(heap->isolate(), this);
- V8::GetCurrentPlatform()->CallOnForegroundThread(isolate, task);
+ auto taskrunner =
+ V8::GetCurrentPlatform()->GetForegroundTaskRunner(isolate);
+ taskrunner->PostTask(base::make_unique<Task>(heap->isolate(), this));
}
}
diff --git a/deps/v8/src/heap/incremental-marking.cc b/deps/v8/src/heap/incremental-marking.cc
index a58d25fff4..239f416eaf 100644
--- a/deps/v8/src/heap/incremental-marking.cc
+++ b/deps/v8/src/heap/incremental-marking.cc
@@ -458,13 +458,6 @@ void IncrementalMarking::FinishBlackAllocation() {
}
}
-void IncrementalMarking::AbortBlackAllocation() {
- if (FLAG_trace_incremental_marking) {
- heap()->isolate()->PrintWithTimestamp(
- "[IncrementalMarking] Black allocation aborted\n");
- }
-}
-
void IncrementalMarking::MarkRoots() {
DCHECK(!finalize_marking_completed_);
DCHECK(IsMarking());
@@ -494,7 +487,6 @@ void IncrementalMarking::RetainMaps() {
// - memory pressure (reduce_memory_footprint_),
// - GC is requested by tests or dev-tools (abort_incremental_marking_).
bool map_retaining_is_disabled = heap()->ShouldReduceMemory() ||
- heap()->ShouldAbortIncrementalMarking() ||
FLAG_retain_maps_for_n_gc == 0;
WeakArrayList* retained_maps = heap()->retained_maps();
int length = retained_maps->length();
@@ -505,10 +497,10 @@ void IncrementalMarking::RetainMaps() {
for (int i = 0; i < length; i += 2) {
MaybeObject* value = retained_maps->Get(i);
HeapObject* map_heap_object;
- if (!value->ToWeakHeapObject(&map_heap_object)) {
+ if (!value->GetHeapObjectIfWeak(&map_heap_object)) {
continue;
}
- int age = Smi::ToInt(retained_maps->Get(i + 1)->ToSmi());
+ int age = Smi::ToInt(retained_maps->Get(i + 1)->cast<Smi>());
int new_age;
Map* map = Map::cast(map_heap_object);
if (i >= number_of_disposed_maps && !map_retaining_is_disabled &&
@@ -801,6 +793,32 @@ intptr_t IncrementalMarking::ProcessMarkingWorklist(
return bytes_processed;
}
+void IncrementalMarking::EmbedderStep(double duration_ms) {
+ constexpr int kObjectsToProcessBeforeInterrupt = 100;
+
+ TRACE_GC(heap()->tracer(), GCTracer::Scope::MC_INCREMENTAL_WRAPPER_TRACING);
+
+ const double deadline =
+ heap_->MonotonicallyIncreasingTimeInMs() + duration_ms;
+
+ HeapObject* object;
+ int cnt = 0;
+ while (marking_worklist()->embedder()->Pop(0, &object)) {
+ heap_->TracePossibleWrapper(JSObject::cast(object));
+ if (++cnt == kObjectsToProcessBeforeInterrupt) {
+ cnt = 0;
+ if (heap_->MonotonicallyIncreasingTimeInMs() > deadline) {
+ break;
+ }
+ }
+ }
+
+ heap_->local_embedder_heap_tracer()->RegisterWrappersWithRemoteTracer();
+ if (!heap_->local_embedder_heap_tracer()
+ ->ShouldFinalizeIncrementalMarking()) {
+ heap_->local_embedder_heap_tracer()->Trace(deadline);
+ }
+}
void IncrementalMarking::Hurry() {
// A scavenge may have pushed new objects on the marking deque (due to black
@@ -930,14 +948,7 @@ double IncrementalMarking::AdvanceIncrementalMarking(
heap_->local_embedder_heap_tracer()->InUse();
do {
if (incremental_wrapper_tracing && trace_wrappers_toggle_) {
- TRACE_GC(heap()->tracer(),
- GCTracer::Scope::MC_INCREMENTAL_WRAPPER_TRACING);
- const double wrapper_deadline =
- heap_->MonotonicallyIncreasingTimeInMs() + kStepSizeInMs;
- if (!heap_->local_embedder_heap_tracer()
- ->ShouldFinalizeIncrementalMarking()) {
- heap_->local_embedder_heap_tracer()->Trace(wrapper_deadline);
- }
+ EmbedderStep(kStepSizeInMs);
} else {
Step(step_size_in_bytes, completion_action, step_origin);
}
diff --git a/deps/v8/src/heap/incremental-marking.h b/deps/v8/src/heap/incremental-marking.h
index 0fb5e11651..ee774c230f 100644
--- a/deps/v8/src/heap/incremental-marking.h
+++ b/deps/v8/src/heap/incremental-marking.h
@@ -100,23 +100,7 @@ class V8_EXPORT_PRIVATE IncrementalMarking {
void NotifyLeftTrimming(HeapObject* from, HeapObject* to);
- V8_INLINE void TransferColor(HeapObject* from, HeapObject* to) {
- if (atomic_marking_state()->IsBlack(to)) {
- DCHECK(black_allocation());
- return;
- }
-
- DCHECK(atomic_marking_state()->IsWhite(to));
- if (atomic_marking_state()->IsGrey(from)) {
- bool success = atomic_marking_state()->WhiteToGrey(to);
- DCHECK(success);
- USE(success);
- } else if (atomic_marking_state()->IsBlack(from)) {
- bool success = atomic_marking_state()->WhiteToBlack(to);
- DCHECK(success);
- USE(success);
- }
- }
+ V8_INLINE void TransferColor(HeapObject* from, HeapObject* to);
State state() const {
DCHECK(state_ == STOPPED || FLAG_incremental_marking);
@@ -193,6 +177,7 @@ class V8_EXPORT_PRIVATE IncrementalMarking {
size_t Step(size_t bytes_to_process, CompletionAction action,
StepOrigin step_origin,
WorklistToProcess worklist_to_process = WorklistToProcess::kAll);
+ void EmbedderStep(double duration);
inline void RestartIfNotMarking();
@@ -248,8 +233,6 @@ class V8_EXPORT_PRIVATE IncrementalMarking {
}
}
- void AbortBlackAllocation();
-
MarkCompactCollector::MarkingWorklist* marking_worklist() const {
return marking_worklist_;
}
diff --git a/deps/v8/src/heap/item-parallel-job.cc b/deps/v8/src/heap/item-parallel-job.cc
index e909ef69d7..b536ccc5d4 100644
--- a/deps/v8/src/heap/item-parallel-job.cc
+++ b/deps/v8/src/heap/item-parallel-job.cc
@@ -58,7 +58,7 @@ ItemParallelJob::~ItemParallelJob() {
}
}
-void ItemParallelJob::Run(std::shared_ptr<Counters> async_counters) {
+void ItemParallelJob::Run(const std::shared_ptr<Counters>& async_counters) {
DCHECK_GT(tasks_.size(), 0);
const size_t num_items = items_.size();
const size_t num_tasks = tasks_.size();
diff --git a/deps/v8/src/heap/item-parallel-job.h b/deps/v8/src/heap/item-parallel-job.h
index 1ad9d22fa4..15351d5d84 100644
--- a/deps/v8/src/heap/item-parallel-job.h
+++ b/deps/v8/src/heap/item-parallel-job.h
@@ -71,7 +71,7 @@ class V8_EXPORT_PRIVATE ItemParallelJob {
class V8_EXPORT_PRIVATE Task : public CancelableTask {
public:
explicit Task(Isolate* isolate);
- virtual ~Task();
+ ~Task() override;
virtual void RunInParallel() = 0;
@@ -137,7 +137,7 @@ class V8_EXPORT_PRIVATE ItemParallelJob {
// Runs this job. Reporting metrics in a thread-safe manner to
// |async_counters|.
- void Run(std::shared_ptr<Counters> async_counters);
+ void Run(const std::shared_ptr<Counters>& async_counters);
private:
std::vector<Item*> items_;
diff --git a/deps/v8/src/heap/mark-compact-inl.h b/deps/v8/src/heap/mark-compact-inl.h
index 466a89080b..449ca43e50 100644
--- a/deps/v8/src/heap/mark-compact-inl.h
+++ b/deps/v8/src/heap/mark-compact-inl.h
@@ -14,6 +14,28 @@
namespace v8 {
namespace internal {
+template <typename ConcreteState, AccessMode access_mode>
+bool MarkingStateBase<ConcreteState, access_mode>::GreyToBlack(
+ HeapObject* obj) {
+ MemoryChunk* p = MemoryChunk::FromAddress(obj->address());
+ MarkBit markbit = MarkBitFrom(p, obj->address());
+ if (!Marking::GreyToBlack<access_mode>(markbit)) return false;
+ static_cast<ConcreteState*>(this)->IncrementLiveBytes(p, obj->Size());
+ return true;
+}
+
+template <typename ConcreteState, AccessMode access_mode>
+bool MarkingStateBase<ConcreteState, access_mode>::WhiteToGrey(
+ HeapObject* obj) {
+ return Marking::WhiteToGrey<access_mode>(MarkBitFrom(obj));
+}
+
+template <typename ConcreteState, AccessMode access_mode>
+bool MarkingStateBase<ConcreteState, access_mode>::WhiteToBlack(
+ HeapObject* obj) {
+ return WhiteToGrey(obj) && GreyToBlack(obj);
+}
+
template <FixedArrayVisitationMode fixed_array_mode,
TraceRetainingPathMode retaining_path_mode, typename MarkingState>
MarkingVisitor<fixed_array_mode, retaining_path_mode,
@@ -26,16 +48,6 @@ MarkingVisitor<fixed_array_mode, retaining_path_mode,
template <FixedArrayVisitationMode fixed_array_mode,
TraceRetainingPathMode retaining_path_mode, typename MarkingState>
int MarkingVisitor<fixed_array_mode, retaining_path_mode,
- MarkingState>::VisitAllocationSite(Map* map,
- AllocationSite* object) {
- int size = AllocationSite::BodyDescriptorWeak::SizeOf(map, object);
- AllocationSite::BodyDescriptorWeak::IterateBody(map, object, size, this);
- return size;
-}
-
-template <FixedArrayVisitationMode fixed_array_mode,
- TraceRetainingPathMode retaining_path_mode, typename MarkingState>
-int MarkingVisitor<fixed_array_mode, retaining_path_mode,
MarkingState>::VisitBytecodeArray(Map* map,
BytecodeArray* array) {
int size = BytecodeArray::BodyDescriptor::SizeOf(map, array);
@@ -46,15 +58,6 @@ int MarkingVisitor<fixed_array_mode, retaining_path_mode,
template <FixedArrayVisitationMode fixed_array_mode,
TraceRetainingPathMode retaining_path_mode, typename MarkingState>
-int MarkingVisitor<fixed_array_mode, retaining_path_mode, MarkingState>::
- VisitCodeDataContainer(Map* map, CodeDataContainer* object) {
- int size = CodeDataContainer::BodyDescriptorWeak::SizeOf(map, object);
- CodeDataContainer::BodyDescriptorWeak::IterateBody(map, object, size, this);
- return size;
-}
-
-template <FixedArrayVisitationMode fixed_array_mode,
- TraceRetainingPathMode retaining_path_mode, typename MarkingState>
int MarkingVisitor<fixed_array_mode, retaining_path_mode,
MarkingState>::VisitFixedArray(Map* map,
FixedArray* object) {
@@ -65,25 +68,48 @@ int MarkingVisitor<fixed_array_mode, retaining_path_mode,
template <FixedArrayVisitationMode fixed_array_mode,
TraceRetainingPathMode retaining_path_mode, typename MarkingState>
-int MarkingVisitor<fixed_array_mode, retaining_path_mode,
- MarkingState>::VisitJSApiObject(Map* map, JSObject* object) {
+template <typename T>
+V8_INLINE int
+MarkingVisitor<fixed_array_mode, retaining_path_mode,
+ MarkingState>::VisitEmbedderTracingSubclass(Map* map,
+ T* object) {
if (heap_->local_embedder_heap_tracer()->InUse()) {
- DCHECK(object->IsJSObject());
heap_->TracePossibleWrapper(object);
}
- int size = JSObject::BodyDescriptor::SizeOf(map, object);
- JSObject::BodyDescriptor::IterateBody(map, object, size, this);
+ int size = T::BodyDescriptor::SizeOf(map, object);
+ T::BodyDescriptor::IterateBody(map, object, size, this);
return size;
}
template <FixedArrayVisitationMode fixed_array_mode,
TraceRetainingPathMode retaining_path_mode, typename MarkingState>
int MarkingVisitor<fixed_array_mode, retaining_path_mode,
- MarkingState>::VisitJSFunction(Map* map,
- JSFunction* object) {
- int size = JSFunction::BodyDescriptorWeak::SizeOf(map, object);
- JSFunction::BodyDescriptorWeak::IterateBody(map, object, size, this);
- return size;
+ MarkingState>::VisitJSApiObject(Map* map, JSObject* object) {
+ return VisitEmbedderTracingSubclass(map, object);
+}
+
+template <FixedArrayVisitationMode fixed_array_mode,
+ TraceRetainingPathMode retaining_path_mode, typename MarkingState>
+int MarkingVisitor<fixed_array_mode, retaining_path_mode,
+ MarkingState>::VisitJSArrayBuffer(Map* map,
+ JSArrayBuffer* object) {
+ return VisitEmbedderTracingSubclass(map, object);
+}
+
+template <FixedArrayVisitationMode fixed_array_mode,
+ TraceRetainingPathMode retaining_path_mode, typename MarkingState>
+int MarkingVisitor<fixed_array_mode, retaining_path_mode,
+ MarkingState>::VisitJSDataView(Map* map,
+ JSDataView* object) {
+ return VisitEmbedderTracingSubclass(map, object);
+}
+
+template <FixedArrayVisitationMode fixed_array_mode,
+ TraceRetainingPathMode retaining_path_mode, typename MarkingState>
+int MarkingVisitor<fixed_array_mode, retaining_path_mode,
+ MarkingState>::VisitJSTypedArray(Map* map,
+ JSTypedArray* object) {
+ return VisitEmbedderTracingSubclass(map, object);
}
template <FixedArrayVisitationMode fixed_array_mode,
@@ -141,16 +167,6 @@ int MarkingVisitor<fixed_array_mode, retaining_path_mode,
template <FixedArrayVisitationMode fixed_array_mode,
TraceRetainingPathMode retaining_path_mode, typename MarkingState>
int MarkingVisitor<fixed_array_mode, retaining_path_mode,
- MarkingState>::VisitNativeContext(Map* map,
- Context* context) {
- int size = Context::BodyDescriptorWeak::SizeOf(map, context);
- Context::BodyDescriptorWeak::IterateBody(map, context, size, this);
- return size;
-}
-
-template <FixedArrayVisitationMode fixed_array_mode,
- TraceRetainingPathMode retaining_path_mode, typename MarkingState>
-int MarkingVisitor<fixed_array_mode, retaining_path_mode,
MarkingState>::VisitTransitionArray(Map* map,
TransitionArray* array) {
int size = TransitionArray::BodyDescriptor::SizeOf(map, array);
@@ -175,11 +191,11 @@ void MarkingVisitor<fixed_array_mode, retaining_path_mode,
MarkingState>::VisitPointer(HeapObject* host,
MaybeObject** p) {
HeapObject* target_object;
- if ((*p)->ToStrongHeapObject(&target_object)) {
+ if ((*p)->GetHeapObjectIfStrong(&target_object)) {
collector_->RecordSlot(host, reinterpret_cast<HeapObjectReference**>(p),
target_object);
MarkObject(host, target_object);
- } else if ((*p)->ToWeakHeapObject(&target_object)) {
+ } else if ((*p)->GetHeapObjectIfWeak(&target_object)) {
if (marking_state()->IsBlackOrGrey(target_object)) {
// Weak references with live values are directly processed here to reduce
// the processing time of weak cells during the main GC pause.
diff --git a/deps/v8/src/heap/mark-compact.cc b/deps/v8/src/heap/mark-compact.cc
index dea105943a..6f46bc57bf 100644
--- a/deps/v8/src/heap/mark-compact.cc
+++ b/deps/v8/src/heap/mark-compact.cc
@@ -17,7 +17,7 @@
#include "src/heap/array-buffer-collector.h"
#include "src/heap/array-buffer-tracker-inl.h"
#include "src/heap/gc-tracer.h"
-#include "src/heap/incremental-marking.h"
+#include "src/heap/incremental-marking-inl.h"
#include "src/heap/invalidated-slots-inl.h"
#include "src/heap/item-parallel-job.h"
#include "src/heap/local-allocator-inl.h"
@@ -193,7 +193,7 @@ class FullMarkingVerifier : public MarkingVerifier {
void VerifyPointers(MaybeObject** start, MaybeObject** end) override {
for (MaybeObject** current = start; current < end; current++) {
HeapObject* object;
- if ((*current)->ToStrongHeapObject(&object)) {
+ if ((*current)->GetHeapObjectIfStrong(&object)) {
CHECK(marking_state_->IsBlackOrGrey(object));
}
}
@@ -309,7 +309,7 @@ class FullEvacuationVerifier : public EvacuationVerifier {
void VerifyPointers(MaybeObject** start, MaybeObject** end) override {
for (MaybeObject** current = start; current < end; current++) {
HeapObject* object;
- if ((*current)->ToStrongHeapObject(&object)) {
+ if ((*current)->GetHeapObjectIfStrong(&object)) {
if (Heap::InNewSpace(object)) {
CHECK(Heap::InToSpace(object));
}
@@ -514,27 +514,6 @@ void MarkCompactCollector::VerifyMarkbitsAreClean() {
#endif // VERIFY_HEAP
-void MarkCompactCollector::ClearMarkbitsInPagedSpace(PagedSpace* space) {
- for (Page* p : *space) {
- non_atomic_marking_state()->ClearLiveness(p);
- }
-}
-
-void MarkCompactCollector::ClearMarkbitsInNewSpace(NewSpace* space) {
- for (Page* p : *space) {
- non_atomic_marking_state()->ClearLiveness(p);
- }
-}
-
-
-void MarkCompactCollector::ClearMarkbits() {
- ClearMarkbitsInPagedSpace(heap_->code_space());
- ClearMarkbitsInPagedSpace(heap_->map_space());
- ClearMarkbitsInPagedSpace(heap_->old_space());
- ClearMarkbitsInNewSpace(heap_->new_space());
- heap_->lo_space()->ClearMarkingStateOfLiveObjects();
-}
-
void MarkCompactCollector::EnsureSweepingCompleted() {
if (!sweeper()->sweeping_in_progress()) return;
@@ -773,20 +752,6 @@ void MarkCompactCollector::Prepare() {
heap()->memory_allocator()->unmapper()->PrepareForMarkCompact();
- // Clear marking bits if incremental marking is aborted.
- if (was_marked_incrementally_ && heap_->ShouldAbortIncrementalMarking()) {
- heap()->incremental_marking()->Stop();
- heap()->incremental_marking()->AbortBlackAllocation();
- FinishConcurrentMarking(ConcurrentMarking::StopRequest::PREEMPT_TASKS);
- heap()->incremental_marking()->Deactivate();
- ClearMarkbits();
- AbortWeakObjects();
- AbortCompaction();
- heap_->local_embedder_heap_tracer()->AbortTracing();
- marking_worklist()->Clear();
- was_marked_incrementally_ = false;
- }
-
if (!was_marked_incrementally_) {
TRACE_GC(heap()->tracer(), GCTracer::Scope::MC_MARK_WRAPPER_PROLOGUE);
heap_->local_embedder_heap_tracer()->TracePrologue();
@@ -1024,7 +989,7 @@ class MarkCompactWeakObjectRetainer : public WeakObjectRetainer {
MarkCompactCollector::NonAtomicMarkingState* marking_state)
: marking_state_(marking_state) {}
- virtual Object* RetainAs(Object* object) {
+ Object* RetainAs(Object* object) override {
HeapObject* heap_object = HeapObject::cast(object);
DCHECK(!marking_state_->IsGrey(heap_object));
if (marking_state_->IsBlack(heap_object)) {
@@ -1112,7 +1077,7 @@ class RecordMigratedSlotVisitor : public ObjectVisitor {
protected:
inline virtual void RecordMigratedSlot(HeapObject* host, MaybeObject* value,
Address slot) {
- if (value->IsStrongOrWeakHeapObject()) {
+ if (value->IsStrongOrWeak()) {
Page* p = Page::FromAddress(reinterpret_cast<Address>(value));
if (p->InNewSpace()) {
DCHECK_IMPLIES(p->InToSpace(),
@@ -1133,7 +1098,7 @@ class MigrationObserver {
public:
explicit MigrationObserver(Heap* heap) : heap_(heap) {}
- virtual ~MigrationObserver() {}
+ virtual ~MigrationObserver() = default;
virtual void Move(AllocationSpace dest, HeapObject* src, HeapObject* dst,
int size) = 0;
@@ -1157,7 +1122,7 @@ class ProfilingMigrationObserver final : public MigrationObserver {
class HeapObjectVisitor {
public:
- virtual ~HeapObjectVisitor() {}
+ virtual ~HeapObjectVisitor() = default;
virtual bool Visit(HeapObject* object, int size) = 0;
};
@@ -1387,7 +1352,7 @@ class EvacuateNewSpacePageVisitor final : public HeapObjectVisitor {
}
}
- inline bool Visit(HeapObject* object, int size) {
+ inline bool Visit(HeapObject* object, int size) override {
if (mode == NEW_TO_NEW) {
heap_->UpdateAllocationSite(object->map(), object,
local_pretenuring_feedback_);
@@ -1429,7 +1394,7 @@ class EvacuateRecordOnlyVisitor final : public HeapObjectVisitor {
public:
explicit EvacuateRecordOnlyVisitor(Heap* heap) : heap_(heap) {}
- inline bool Visit(HeapObject* object, int size) {
+ inline bool Visit(HeapObject* object, int size) override {
RecordMigratedSlotVisitor visitor(heap_->mark_compact_collector());
object->IterateBodyFast(&visitor);
return true;
@@ -1633,6 +1598,10 @@ void MarkCompactCollector::ProcessEphemeronsLinear() {
void MarkCompactCollector::PerformWrapperTracing() {
if (heap_->local_embedder_heap_tracer()->InUse()) {
TRACE_GC(heap()->tracer(), GCTracer::Scope::MC_MARK_WRAPPER_TRACING);
+ HeapObject* object;
+ while (marking_worklist()->embedder()->Pop(kMainThread, &object)) {
+ heap_->TracePossibleWrapper(JSObject::cast(object));
+ }
heap_->local_embedder_heap_tracer()->RegisterWrappersWithRemoteTracer();
heap_->local_embedder_heap_tracer()->Trace(
std::numeric_limits<double>::infinity());
@@ -1788,10 +1757,14 @@ void MarkCompactCollector::MarkLiveObjects() {
// through ephemerons.
{
TRACE_GC(heap()->tracer(), GCTracer::Scope::MC_MARK_WRAPPERS);
- while (!heap_->local_embedder_heap_tracer()->IsRemoteTracingDone()) {
+ do {
+ // PerformWrapperTracing() also empties the work items collected by
+ // concurrent markers. As a result this call needs to happen at least
+ // once.
PerformWrapperTracing();
ProcessMarkingWorklist();
- }
+ } while (!heap_->local_embedder_heap_tracer()->IsRemoteTracingDone());
+ DCHECK(marking_worklist()->IsEmbedderEmpty());
DCHECK(marking_worklist()->IsEmpty());
}
@@ -1838,6 +1811,7 @@ void MarkCompactCollector::MarkLiveObjects() {
TRACE_GC(heap()->tracer(), GCTracer::Scope::MC_MARK_WRAPPER_EPILOGUE);
heap()->local_embedder_heap_tracer()->TraceEpilogue();
}
+ DCHECK(marking_worklist()->IsEmbedderEmpty());
DCHECK(marking_worklist()->IsEmpty());
}
@@ -2092,7 +2066,7 @@ void MarkCompactCollector::ClearWeakReferences() {
while (weak_objects_.weak_references.Pop(kMainThread, &slot)) {
HeapObject* value;
HeapObjectReference** location = slot.second;
- if ((*location)->ToWeakHeapObject(&value)) {
+ if ((*location)->GetHeapObjectIfWeak(&value)) {
DCHECK(!value->IsCell());
if (non_atomic_marking_state()->IsBlackOrGrey(value)) {
// The value of the weak reference is alive.
@@ -2174,9 +2148,9 @@ template <AccessMode access_mode>
static inline SlotCallbackResult UpdateSlot(MaybeObject** slot) {
MaybeObject* obj = base::AsAtomicPointer::Relaxed_Load(slot);
HeapObject* heap_obj;
- if (obj->ToWeakHeapObject(&heap_obj)) {
+ if (obj->GetHeapObjectIfWeak(&heap_obj)) {
UpdateSlot<access_mode>(slot, obj, heap_obj, HeapObjectReferenceType::WEAK);
- } else if (obj->ToStrongHeapObject(&heap_obj)) {
+ } else if (obj->GetHeapObjectIfStrong(&heap_obj)) {
return UpdateSlot<access_mode>(slot, obj, heap_obj,
HeapObjectReferenceType::STRONG);
}
@@ -2185,7 +2159,7 @@ static inline SlotCallbackResult UpdateSlot(MaybeObject** slot) {
template <AccessMode access_mode>
static inline SlotCallbackResult UpdateStrongSlot(MaybeObject** maybe_slot) {
- DCHECK((*maybe_slot)->IsSmi() || (*maybe_slot)->IsStrongHeapObject());
+ DCHECK((*maybe_slot)->IsSmi() || (*maybe_slot)->IsStrong());
Object** slot = reinterpret_cast<Object**>(maybe_slot);
Object* obj = base::AsAtomicPointer::Relaxed_Load(slot);
if (obj->IsHeapObject()) {
@@ -2248,8 +2222,7 @@ class PointersUpdatingVisitor : public ObjectVisitor, public RootVisitor {
private:
static inline SlotCallbackResult UpdateStrongMaybeObjectSlotInternal(
MaybeObject** slot) {
- DCHECK(!(*slot)->IsWeakHeapObject());
- DCHECK(!(*slot)->IsClearedWeakHeapObject());
+ DCHECK(!(*slot)->IsWeakOrCleared());
return UpdateStrongSlotInternal(reinterpret_cast<Object**>(slot));
}
@@ -2274,9 +2247,11 @@ static String* UpdateReferenceInExternalStringTableEntry(Heap* heap,
String* new_string = String::cast(map_word.ToForwardingAddress());
if (new_string->IsExternalString()) {
- heap->ProcessMovedExternalString(
+ MemoryChunk::MoveExternalBackingStoreBytes(
+ ExternalBackingStoreType::kExternalString,
Page::FromAddress(reinterpret_cast<Address>(*p)),
- Page::FromHeapObject(new_string), ExternalString::cast(new_string));
+ Page::FromHeapObject(new_string),
+ ExternalString::cast(new_string)->ExternalPayloadSize());
}
return new_string;
}
@@ -2364,7 +2339,7 @@ class Evacuator : public Malloced {
duration_(0.0),
bytes_compacted_(0) {}
- virtual ~Evacuator() {}
+ virtual ~Evacuator() = default;
void EvacuatePage(Page* page);
@@ -2520,7 +2495,7 @@ void FullEvacuator::RawEvacuatePage(Page* page, intptr_t* live_bytes) {
class PageEvacuationItem : public ItemParallelJob::Item {
public:
explicit PageEvacuationItem(Page* page) : page_(page) {}
- virtual ~PageEvacuationItem() {}
+ ~PageEvacuationItem() override = default;
Page* page() const { return page_; }
private:
@@ -2559,11 +2534,7 @@ void MarkCompactCollectorBase::CreateAndExecuteEvacuationTasks(
compaction_speed = heap()->tracer()->CompactionSpeedInBytesPerMillisecond();
}
- const bool profiling =
- heap()->isolate()->is_profiling() ||
- heap()->isolate()->logger()->is_listening_to_code_events() ||
- heap()->isolate()->heap_profiler()->is_tracking_object_moves() ||
- heap()->has_heap_object_allocation_tracker();
+ const bool profiling = isolate()->LogObjectRelocation();
ProfilingMigrationObserver profiling_observer(heap());
const int wanted_num_tasks =
@@ -2642,7 +2613,7 @@ void MarkCompactCollector::EvacuatePagesInParallel() {
class EvacuationWeakObjectRetainer : public WeakObjectRetainer {
public:
- virtual Object* RetainAs(Object* object) {
+ Object* RetainAs(Object* object) override {
if (object->IsHeapObject()) {
HeapObject* heap_object = HeapObject::cast(object);
MapWord map_word = heap_object->map_word();
@@ -2817,7 +2788,7 @@ void MarkCompactCollector::Evacuate() {
class UpdatingItem : public ItemParallelJob::Item {
public:
- virtual ~UpdatingItem() {}
+ ~UpdatingItem() override = default;
virtual void Process() = 0;
};
@@ -2852,7 +2823,7 @@ class ToSpaceUpdatingItem : public UpdatingItem {
start_(start),
end_(end),
marking_state_(marking_state) {}
- virtual ~ToSpaceUpdatingItem() {}
+ ~ToSpaceUpdatingItem() override = default;
void Process() override {
if (chunk_->IsFlagSet(Page::PAGE_NEW_NEW_PROMOTION)) {
@@ -2906,7 +2877,7 @@ class RememberedSetUpdatingItem : public UpdatingItem {
marking_state_(marking_state),
chunk_(chunk),
updating_mode_(updating_mode) {}
- virtual ~RememberedSetUpdatingItem() {}
+ ~RememberedSetUpdatingItem() override = default;
void Process() override {
TRACE_EVENT0(TRACE_DISABLED_BY_DEFAULT("v8.gc"),
@@ -2921,7 +2892,7 @@ class RememberedSetUpdatingItem : public UpdatingItem {
inline SlotCallbackResult CheckAndUpdateOldToNewSlot(Address slot_address) {
MaybeObject** slot = reinterpret_cast<MaybeObject**>(slot_address);
HeapObject* heap_object;
- if (!(*slot)->ToStrongOrWeakHeapObject(&heap_object)) {
+ if (!(*slot)->GetHeapObject(&heap_object)) {
return REMOVE_SLOT;
}
if (Heap::InFromSpace(heap_object)) {
@@ -2931,7 +2902,7 @@ class RememberedSetUpdatingItem : public UpdatingItem {
reinterpret_cast<HeapObjectReference**>(slot),
map_word.ToForwardingAddress());
}
- bool success = (*slot)->ToStrongOrWeakHeapObject(&heap_object);
+ bool success = (*slot)->GetHeapObject(&heap_object);
USE(success);
DCHECK(success);
// If the object was in from space before and is after executing the
@@ -3054,7 +3025,7 @@ class GlobalHandlesUpdatingItem : public UpdatingItem {
global_handles_(global_handles),
start_(start),
end_(end) {}
- virtual ~GlobalHandlesUpdatingItem() {}
+ ~GlobalHandlesUpdatingItem() override = default;
void Process() override {
TRACE_EVENT0(TRACE_DISABLED_BY_DEFAULT("v8.gc"),
@@ -3081,7 +3052,7 @@ class ArrayBufferTrackerUpdatingItem : public UpdatingItem {
explicit ArrayBufferTrackerUpdatingItem(Page* page, EvacuationState state)
: page_(page), state_(state) {}
- virtual ~ArrayBufferTrackerUpdatingItem() {}
+ ~ArrayBufferTrackerUpdatingItem() override = default;
void Process() override {
TRACE_EVENT1(TRACE_DISABLED_BY_DEFAULT("v8.gc"),
@@ -3255,7 +3226,7 @@ void MarkCompactCollector::UpdatePointersAfterEvacuation() {
GCTracer::BackgroundScope::MC_BACKGROUND_EVACUATE_UPDATE_POINTERS));
}
updating_job.Run(isolate()->async_counters());
- heap()->array_buffer_collector()->FreeAllocationsOnBackgroundThread();
+ heap()->array_buffer_collector()->FreeAllocations();
}
}
@@ -3426,8 +3397,9 @@ void MarkCompactCollector::MarkingWorklist::PrintWorklist(
count[obj->map()->instance_type()]++;
});
std::vector<std::pair<int, InstanceType>> rank;
- for (auto i : count) {
- rank.push_back(std::make_pair(i.second, i.first));
+ rank.reserve(count.size());
+ for (const auto& i : count) {
+ rank.emplace_back(i.second, i.first);
}
std::map<InstanceType, std::string> instance_type_name;
#define INSTANCE_TYPE_NAME(name) instance_type_name[name] = #name;
@@ -3486,7 +3458,7 @@ class YoungGenerationMarkingVerifier : public MarkingVerifier {
for (MaybeObject** current = start; current < end; current++) {
HeapObject* object;
// Minor MC treats weak references as strong.
- if ((*current)->ToStrongOrWeakHeapObject(&object)) {
+ if ((*current)->GetHeapObject(&object)) {
if (!Heap::InNewSpace(object)) {
continue;
}
@@ -3524,7 +3496,7 @@ class YoungGenerationEvacuationVerifier : public EvacuationVerifier {
void VerifyPointers(MaybeObject** start, MaybeObject** end) override {
for (MaybeObject** current = start; current < end; current++) {
HeapObject* object;
- if ((*current)->ToStrongOrWeakHeapObject(&object)) {
+ if ((*current)->GetHeapObject(&object)) {
CHECK_IMPLIES(Heap::InNewSpace(object), Heap::InToSpace(object));
}
}
@@ -3593,7 +3565,7 @@ class YoungGenerationMarkingVisitor final
HeapObject* target_object;
// Treat weak references as strong. TODO(marja): Proper weakness handling
// for minor-mcs.
- if (target->ToStrongOrWeakHeapObject(&target_object)) {
+ if (target->GetHeapObject(&target_object)) {
MarkObjectViaMarkingWorklist(target_object);
}
}
@@ -3697,7 +3669,7 @@ class YoungGenerationRecordMigratedSlotVisitor final
inline void RecordMigratedSlot(HeapObject* host, MaybeObject* value,
Address slot) final {
- if (value->IsStrongOrWeakHeapObject()) {
+ if (value->IsStrongOrWeak()) {
Page* p = Page::FromAddress(reinterpret_cast<Address>(value));
if (p->InNewSpace()) {
DCHECK_IMPLIES(p->InToSpace(),
@@ -3758,7 +3730,7 @@ void MinorMarkCompactCollector::UpdatePointersAfterEvacuation() {
TRACE_GC(heap()->tracer(),
GCTracer::Scope::MINOR_MC_EVACUATE_UPDATE_POINTERS_SLOTS);
updating_job.Run(isolate()->async_counters());
- heap()->array_buffer_collector()->FreeAllocationsOnBackgroundThread();
+ heap()->array_buffer_collector()->FreeAllocations();
}
{
@@ -3949,7 +3921,7 @@ class MinorMarkCompactWeakObjectRetainer : public WeakObjectRetainer {
MinorMarkCompactCollector* collector)
: marking_state_(collector->non_atomic_marking_state()) {}
- virtual Object* RetainAs(Object* object) {
+ Object* RetainAs(Object* object) override {
HeapObject* heap_object = HeapObject::cast(object);
if (!Heap::InNewSpace(heap_object)) return object;
@@ -4024,7 +3996,7 @@ class YoungGenerationMarkingTask;
class MarkingItem : public ItemParallelJob::Item {
public:
- virtual ~MarkingItem() {}
+ ~MarkingItem() override = default;
virtual void Process(YoungGenerationMarkingTask* task) = 0;
};
@@ -4112,7 +4084,7 @@ class PageMarkingItem : public MarkingItem {
public:
explicit PageMarkingItem(MemoryChunk* chunk, std::atomic<int>* global_slots)
: chunk_(chunk), global_slots_(global_slots), slots_(0) {}
- virtual ~PageMarkingItem() { *global_slots_ = *global_slots_ + slots_; }
+ ~PageMarkingItem() override { *global_slots_ = *global_slots_ + slots_; }
void Process(YoungGenerationMarkingTask* task) override {
TRACE_EVENT0(TRACE_DISABLED_BY_DEFAULT("v8.gc"),
@@ -4152,7 +4124,7 @@ class PageMarkingItem : public MarkingItem {
// has to be in ToSpace.
DCHECK(Heap::InToSpace(object));
HeapObject* heap_object;
- bool success = object->ToStrongOrWeakHeapObject(&heap_object);
+ bool success = object->GetHeapObject(&heap_object);
USE(success);
DCHECK(success);
task->MarkObject(heap_object);
@@ -4172,7 +4144,7 @@ class GlobalHandlesMarkingItem : public MarkingItem {
GlobalHandlesMarkingItem(Heap* heap, GlobalHandles* global_handles,
size_t start, size_t end)
: global_handles_(global_handles), start_(start), end_(end) {}
- virtual ~GlobalHandlesMarkingItem() {}
+ ~GlobalHandlesMarkingItem() override = default;
void Process(YoungGenerationMarkingTask* task) override {
TRACE_EVENT0(TRACE_DISABLED_BY_DEFAULT("v8.gc"),
diff --git a/deps/v8/src/heap/mark-compact.h b/deps/v8/src/heap/mark-compact.h
index d62c964336..c4ab5b2b9c 100644
--- a/deps/v8/src/heap/mark-compact.h
+++ b/deps/v8/src/heap/mark-compact.h
@@ -63,21 +63,9 @@ class MarkingStateBase {
return Marking::IsBlackOrGrey<access_mode>(MarkBitFrom(obj));
}
- V8_INLINE bool WhiteToGrey(HeapObject* obj) {
- return Marking::WhiteToGrey<access_mode>(MarkBitFrom(obj));
- }
-
- V8_INLINE bool WhiteToBlack(HeapObject* obj) {
- return WhiteToGrey(obj) && GreyToBlack(obj);
- }
-
- V8_INLINE bool GreyToBlack(HeapObject* obj) {
- MemoryChunk* p = MemoryChunk::FromAddress(obj->address());
- MarkBit markbit = MarkBitFrom(p, obj->address());
- if (!Marking::GreyToBlack<access_mode>(markbit)) return false;
- static_cast<ConcreteState*>(this)->IncrementLiveBytes(p, obj->Size());
- return true;
- }
+ V8_INLINE bool WhiteToGrey(HeapObject* obj);
+ V8_INLINE bool WhiteToBlack(HeapObject* obj);
+ V8_INLINE bool GreyToBlack(HeapObject* obj);
void ClearLiveness(MemoryChunk* chunk) {
static_cast<ConcreteState*>(this)->bitmap(chunk)->Clear();
@@ -250,7 +238,7 @@ enum class RememberedSetUpdatingMode { ALL, OLD_TO_NEW_ONLY };
// Base class for minor and full MC collectors.
class MarkCompactCollectorBase {
public:
- virtual ~MarkCompactCollectorBase() {}
+ virtual ~MarkCompactCollectorBase() = default;
virtual void SetUp() = 0;
virtual void TearDown() = 0;
@@ -464,11 +452,14 @@ class MarkCompactCollector final : public MarkCompactCollectorBase {
#else
using MarkingState = MajorNonAtomicMarkingState;
#endif // V8_CONCURRENT_MARKING
+
using NonAtomicMarkingState = MajorNonAtomicMarkingState;
+
// Wrapper for the shared and bailout worklists.
class MarkingWorklist {
public:
using ConcurrentMarkingWorklist = Worklist<HeapObject*, 64>;
+ using EmbedderTracingWorklist = Worklist<HeapObject*, 16>;
// The heap parameter is not used but needed to match the sequential case.
explicit MarkingWorklist(Heap* heap) {}
@@ -500,8 +491,8 @@ class MarkCompactCollector final : public MarkCompactCollectorBase {
}
HeapObject* PopBailout() {
- HeapObject* result;
#ifdef V8_CONCURRENT_MARKING
+ HeapObject* result;
if (bailout_.Pop(kMainThread, &result)) return result;
#endif
return nullptr;
@@ -511,6 +502,7 @@ class MarkCompactCollector final : public MarkCompactCollectorBase {
bailout_.Clear();
shared_.Clear();
on_hold_.Clear();
+ embedder_.Clear();
}
bool IsBailoutEmpty() { return bailout_.IsLocalEmpty(kMainThread); }
@@ -523,6 +515,11 @@ class MarkCompactCollector final : public MarkCompactCollectorBase {
on_hold_.IsGlobalPoolEmpty();
}
+ bool IsEmbedderEmpty() {
+ return embedder_.IsLocalEmpty(kMainThread) &&
+ embedder_.IsGlobalPoolEmpty();
+ }
+
int Size() {
return static_cast<int>(bailout_.LocalSize(kMainThread) +
shared_.LocalSize(kMainThread) +
@@ -538,11 +535,13 @@ class MarkCompactCollector final : public MarkCompactCollectorBase {
bailout_.Update(callback);
shared_.Update(callback);
on_hold_.Update(callback);
+ embedder_.Update(callback);
}
ConcurrentMarkingWorklist* shared() { return &shared_; }
ConcurrentMarkingWorklist* bailout() { return &bailout_; }
ConcurrentMarkingWorklist* on_hold() { return &on_hold_; }
+ EmbedderTracingWorklist* embedder() { return &embedder_; }
void Print() {
PrintWorklist("shared", &shared_);
@@ -568,6 +567,11 @@ class MarkCompactCollector final : public MarkCompactCollectorBase {
// for new space. This allow the compiler to remove write barriers
// for freshly allocatd objects.
ConcurrentMarkingWorklist on_hold_;
+
+ // Worklist for objects that potentially require embedder tracing, i.e.,
+ // these objects need to be handed over to the embedder to find the full
+ // transitive closure.
+ EmbedderTracingWorklist embedder_;
};
class RootMarkingVisitor;
@@ -626,8 +630,6 @@ class MarkCompactCollector final : public MarkCompactCollectorBase {
void UpdateSlots(SlotsBuffer* buffer);
void UpdateSlotsRecordedIn(SlotsBuffer* buffer);
- void ClearMarkbits();
-
bool is_compacting() const { return compacting_; }
// Ensures that sweeping is finished.
@@ -703,7 +705,7 @@ class MarkCompactCollector final : public MarkCompactCollectorBase {
private:
explicit MarkCompactCollector(Heap* heap);
- ~MarkCompactCollector();
+ ~MarkCompactCollector() override;
bool WillBeDeoptimized(Code* code);
@@ -835,9 +837,6 @@ class MarkCompactCollector final : public MarkCompactCollectorBase {
void PostProcessEvacuationCandidates();
void ReportAbortedEvacuationCandidate(HeapObject* failed_object, Page* page);
- void ClearMarkbitsInPagedSpace(PagedSpace* space);
- void ClearMarkbitsInNewSpace(NewSpace* space);
-
static const int kEphemeronChunkSize = 8 * KB;
int NumberOfParallelEphemeronVisitingTasks(size_t elements);
@@ -910,15 +909,14 @@ class MarkingVisitor final
V8_INLINE bool ShouldVisitMapPointer() { return false; }
- V8_INLINE int VisitAllocationSite(Map* map, AllocationSite* object);
V8_INLINE int VisitBytecodeArray(Map* map, BytecodeArray* object);
- V8_INLINE int VisitCodeDataContainer(Map* map, CodeDataContainer* object);
V8_INLINE int VisitEphemeronHashTable(Map* map, EphemeronHashTable* object);
V8_INLINE int VisitFixedArray(Map* map, FixedArray* object);
V8_INLINE int VisitJSApiObject(Map* map, JSObject* object);
- V8_INLINE int VisitJSFunction(Map* map, JSFunction* object);
+ V8_INLINE int VisitJSArrayBuffer(Map* map, JSArrayBuffer* object);
+ V8_INLINE int VisitJSDataView(Map* map, JSDataView* object);
+ V8_INLINE int VisitJSTypedArray(Map* map, JSTypedArray* object);
V8_INLINE int VisitMap(Map* map, Map* object);
- V8_INLINE int VisitNativeContext(Map* map, Context* object);
V8_INLINE int VisitTransitionArray(Map* map, TransitionArray* object);
// ObjectVisitor implementation.
@@ -931,6 +929,11 @@ class MarkingVisitor final
V8_INLINE void VisitEmbeddedPointer(Code* host, RelocInfo* rinfo) final;
V8_INLINE void VisitCodeTarget(Code* host, RelocInfo* rinfo) final;
+ // Weak list pointers should be ignored during marking. The lists are
+ // reconstructed after GC.
+ void VisitCustomWeakPointers(HeapObject* host, Object** start,
+ Object** end) final {}
+
private:
// Granularity in which FixedArrays are scanned if |fixed_array_mode|
// is true.
@@ -938,6 +941,9 @@ class MarkingVisitor final
V8_INLINE int VisitFixedArrayIncremental(Map* map, FixedArray* object);
+ template <typename T>
+ V8_INLINE int VisitEmbedderTracingSubclass(Map* map, T* object);
+
V8_INLINE void MarkMapContents(Map* map);
// Marks the object black without pushing it on the marking work list. Returns
@@ -980,7 +986,7 @@ class MinorMarkCompactCollector final : public MarkCompactCollectorBase {
using NonAtomicMarkingState = MinorNonAtomicMarkingState;
explicit MinorMarkCompactCollector(Heap* heap);
- ~MinorMarkCompactCollector();
+ ~MinorMarkCompactCollector() override;
MarkingState* marking_state() { return &marking_state_; }
diff --git a/deps/v8/src/heap/marking.cc b/deps/v8/src/heap/marking.cc
index 23fbdd3465..93b5c06a45 100644
--- a/deps/v8/src/heap/marking.cc
+++ b/deps/v8/src/heap/marking.cc
@@ -28,6 +28,9 @@ void Bitmap::MarkAllBits() {
}
void Bitmap::SetRange(uint32_t start_index, uint32_t end_index) {
+ if (start_index >= end_index) return;
+ end_index--;
+
unsigned int start_cell_index = start_index >> Bitmap::kBitsPerCellLog2;
MarkBit::CellType start_index_mask = 1u << Bitmap::IndexInCell(start_index);
unsigned int end_cell_index = end_index >> Bitmap::kBitsPerCellLog2;
@@ -43,10 +46,11 @@ void Bitmap::SetRange(uint32_t start_index, uint32_t end_index) {
base::Relaxed_Store(cell_base + i, ~0u);
}
// Finally, fill all bits until the end address in the last cell with 1s.
- SetBitsInCell<AccessMode::ATOMIC>(end_cell_index, (end_index_mask - 1));
+ SetBitsInCell<AccessMode::ATOMIC>(end_cell_index,
+ end_index_mask | (end_index_mask - 1));
} else {
- SetBitsInCell<AccessMode::ATOMIC>(start_cell_index,
- end_index_mask - start_index_mask);
+ SetBitsInCell<AccessMode::ATOMIC>(
+ start_cell_index, end_index_mask | (end_index_mask - start_index_mask));
}
// This fence prevents re-ordering of publishing stores with the mark-
// bit setting stores.
@@ -54,6 +58,9 @@ void Bitmap::SetRange(uint32_t start_index, uint32_t end_index) {
}
void Bitmap::ClearRange(uint32_t start_index, uint32_t end_index) {
+ if (start_index >= end_index) return;
+ end_index--;
+
unsigned int start_cell_index = start_index >> Bitmap::kBitsPerCellLog2;
MarkBit::CellType start_index_mask = 1u << Bitmap::IndexInCell(start_index);
@@ -71,10 +78,11 @@ void Bitmap::ClearRange(uint32_t start_index, uint32_t end_index) {
base::Relaxed_Store(cell_base + i, 0);
}
// Finally, set all bits until the end address in the last cell with 0s.
- ClearBitsInCell<AccessMode::ATOMIC>(end_cell_index, (end_index_mask - 1));
+ ClearBitsInCell<AccessMode::ATOMIC>(end_cell_index,
+ end_index_mask | (end_index_mask - 1));
} else {
- ClearBitsInCell<AccessMode::ATOMIC>(start_cell_index,
- (end_index_mask - start_index_mask));
+ ClearBitsInCell<AccessMode::ATOMIC>(
+ start_cell_index, end_index_mask | (end_index_mask - start_index_mask));
}
// This fence prevents re-ordering of publishing stores with the mark-
// bit clearing stores.
@@ -82,6 +90,9 @@ void Bitmap::ClearRange(uint32_t start_index, uint32_t end_index) {
}
bool Bitmap::AllBitsSetInRange(uint32_t start_index, uint32_t end_index) {
+ if (start_index >= end_index) return false;
+ end_index--;
+
unsigned int start_cell_index = start_index >> Bitmap::kBitsPerCellLog2;
MarkBit::CellType start_index_mask = 1u << Bitmap::IndexInCell(start_index);
@@ -97,21 +108,18 @@ bool Bitmap::AllBitsSetInRange(uint32_t start_index, uint32_t end_index) {
for (unsigned int i = start_cell_index + 1; i < end_cell_index; i++) {
if (cells()[i] != ~0u) return false;
}
- matching_mask = (end_index_mask - 1);
- // Check against a mask of 0 to avoid dereferencing the cell after the
- // end of the bitmap.
- return (matching_mask == 0) ||
- ((cells()[end_cell_index] & matching_mask) == matching_mask);
+ matching_mask = end_index_mask | (end_index_mask - 1);
+ return ((cells()[end_cell_index] & matching_mask) == matching_mask);
} else {
- matching_mask = end_index_mask - start_index_mask;
- // Check against a mask of 0 to avoid dereferencing the cell after the
- // end of the bitmap.
- return (matching_mask == 0) ||
- (cells()[end_cell_index] & matching_mask) == matching_mask;
+ matching_mask = end_index_mask | (end_index_mask - start_index_mask);
+ return (cells()[end_cell_index] & matching_mask) == matching_mask;
}
}
bool Bitmap::AllBitsClearInRange(uint32_t start_index, uint32_t end_index) {
+ if (start_index >= end_index) return true;
+ end_index--;
+
unsigned int start_cell_index = start_index >> Bitmap::kBitsPerCellLog2;
MarkBit::CellType start_index_mask = 1u << Bitmap::IndexInCell(start_index);
@@ -125,15 +133,11 @@ bool Bitmap::AllBitsClearInRange(uint32_t start_index, uint32_t end_index) {
for (unsigned int i = start_cell_index + 1; i < end_cell_index; i++) {
if (cells()[i]) return false;
}
- matching_mask = (end_index_mask - 1);
- // Check against a mask of 0 to avoid dereferencing the cell after the
- // end of the bitmap.
- return (matching_mask == 0) || !(cells()[end_cell_index] & matching_mask);
+ matching_mask = end_index_mask | (end_index_mask - 1);
+ return !(cells()[end_cell_index] & matching_mask);
} else {
- matching_mask = end_index_mask - start_index_mask;
- // Check against a mask of 0 to avoid dereferencing the cell after the
- // end of the bitmap.
- return (matching_mask == 0) || !(cells()[end_cell_index] & matching_mask);
+ matching_mask = end_index_mask | (end_index_mask - start_index_mask);
+ return !(cells()[end_cell_index] & matching_mask);
}
}
diff --git a/deps/v8/src/heap/object-stats.cc b/deps/v8/src/heap/object-stats.cc
index ac7bcb8087..bb069d19f4 100644
--- a/deps/v8/src/heap/object-stats.cc
+++ b/deps/v8/src/heap/object-stats.cc
@@ -547,7 +547,7 @@ void ObjectStatsCollectorImpl::RecordVirtualJSObjectDetails(JSObject* object) {
static ObjectStats::VirtualInstanceType GetFeedbackSlotType(
MaybeObject* maybe_obj, FeedbackSlotKind kind, Isolate* isolate) {
- if (maybe_obj->IsClearedWeakHeapObject())
+ if (maybe_obj->IsCleared())
return ObjectStats::FEEDBACK_VECTOR_SLOT_OTHER_TYPE;
Object* obj = maybe_obj->GetHeapObjectOrSmi();
switch (kind) {
@@ -623,11 +623,12 @@ void ObjectStatsCollectorImpl::RecordVirtualFeedbackVectorDetails(
// Log the monomorphic/polymorphic helper objects that this slot owns.
for (int i = 0; i < it.entry_size(); i++) {
MaybeObject* raw_object = vector->get(slot.ToInt() + i);
- if (!raw_object->IsStrongOrWeakHeapObject()) continue;
- HeapObject* object = raw_object->GetHeapObject();
- if (object->IsCell() || object->IsWeakFixedArray()) {
- RecordSimpleVirtualObjectStats(
- vector, object, ObjectStats::FEEDBACK_VECTOR_ENTRY_TYPE);
+ HeapObject* object;
+ if (raw_object->GetHeapObject(&object)) {
+ if (object->IsCell() || object->IsWeakFixedArray()) {
+ RecordSimpleVirtualObjectStats(
+ vector, object, ObjectStats::FEEDBACK_VECTOR_ENTRY_TYPE);
+ }
}
}
}
@@ -677,8 +678,6 @@ void ObjectStatsCollectorImpl::CollectStatistics(
RecordVirtualContext(Context::cast(obj));
} else if (obj->IsScript()) {
RecordVirtualScriptDetails(Script::cast(obj));
- } else if (obj->IsExternalString()) {
- RecordVirtualExternalStringDetails(ExternalString::cast(obj));
} else if (obj->IsArrayBoilerplateDescription()) {
RecordVirtualArrayBoilerplateDescription(
ArrayBoilerplateDescription::cast(obj));
@@ -688,6 +687,11 @@ void ObjectStatsCollectorImpl::CollectStatistics(
}
break;
case kPhase2:
+ if (obj->IsExternalString()) {
+ // This has to be in Phase2 to avoid conflicting with recording Script
+ // sources. We still want to run RecordObjectStats after though.
+ RecordVirtualExternalStringDetails(ExternalString::cast(obj));
+ }
RecordObjectStats(obj, map->instance_type(), obj->Size());
if (collect_field_stats == CollectFieldStats::kYes) {
field_stats_collector_.RecordStats(obj);
@@ -808,7 +812,7 @@ void ObjectStatsCollectorImpl::RecordVirtualScriptDetails(Script* script) {
} else if (raw_source->IsString()) {
String* source = String::cast(raw_source);
RecordSimpleVirtualObjectStats(
- script, HeapObject::cast(raw_source),
+ script, source,
source->IsOneByteRepresentation()
? ObjectStats::SCRIPT_SOURCE_NON_EXTERNAL_ONE_BYTE_TYPE
: ObjectStats::SCRIPT_SOURCE_NON_EXTERNAL_TWO_BYTE_TYPE);
diff --git a/deps/v8/src/heap/objects-visiting-inl.h b/deps/v8/src/heap/objects-visiting-inl.h
index f32bbc1914..c7a4f70f01 100644
--- a/deps/v8/src/heap/objects-visiting-inl.h
+++ b/deps/v8/src/heap/objects-visiting-inl.h
@@ -170,15 +170,6 @@ ResultType HeapVisitor<ResultType, ConcreteVisitor>::VisitFreeSpace(
}
template <typename ConcreteVisitor>
-int NewSpaceVisitor<ConcreteVisitor>::VisitJSFunction(Map* map,
- JSFunction* object) {
- ConcreteVisitor* visitor = static_cast<ConcreteVisitor*>(this);
- int size = JSFunction::BodyDescriptorWeak::SizeOf(map, object);
- JSFunction::BodyDescriptorWeak::IterateBody(map, object, size, visitor);
- return size;
-}
-
-template <typename ConcreteVisitor>
int NewSpaceVisitor<ConcreteVisitor>::VisitNativeContext(Map* map,
Context* object) {
ConcreteVisitor* visitor = static_cast<ConcreteVisitor*>(this);
diff --git a/deps/v8/src/heap/objects-visiting.h b/deps/v8/src/heap/objects-visiting.h
index 63ef8fb353..147af52c7e 100644
--- a/deps/v8/src/heap/objects-visiting.h
+++ b/deps/v8/src/heap/objects-visiting.h
@@ -21,7 +21,9 @@ class BigInt;
class BytecodeArray;
class DataHandler;
class JSArrayBuffer;
+class JSDataView;
class JSRegExp;
+class JSTypedArray;
class JSWeakCollection;
class UncompiledDataWithoutPreParsedScope;
class UncompiledDataWithPreParsedScope;
@@ -44,8 +46,9 @@ class UncompiledDataWithPreParsedScope;
V(FixedFloat64Array) \
V(FixedTypedArrayBase) \
V(JSArrayBuffer) \
- V(JSFunction) \
+ V(JSDataView) \
V(JSObject) \
+ V(JSTypedArray) \
V(JSWeakCollection) \
V(Map) \
V(Oddball) \
@@ -119,7 +122,6 @@ class NewSpaceVisitor : public HeapVisitor<int, ConcreteVisitor> {
// Special cases for young generation.
- V8_INLINE int VisitJSFunction(Map* map, JSFunction* object);
V8_INLINE int VisitNativeContext(Map* map, Context* object);
V8_INLINE int VisitJSApiObject(Map* map, JSObject* object);
diff --git a/deps/v8/src/heap/scavenge-job.cc b/deps/v8/src/heap/scavenge-job.cc
index 9feebbf4d5..5848d5342e 100644
--- a/deps/v8/src/heap/scavenge-job.cc
+++ b/deps/v8/src/heap/scavenge-job.cc
@@ -107,8 +107,9 @@ void ScavengeJob::ScheduleIdleTask(Heap* heap) {
v8::Isolate* isolate = reinterpret_cast<v8::Isolate*>(heap->isolate());
if (V8::GetCurrentPlatform()->IdleTasksEnabled(isolate)) {
idle_task_pending_ = true;
- auto task = new IdleTask(heap->isolate(), this);
- V8::GetCurrentPlatform()->CallIdleOnForegroundThread(isolate, task);
+ auto task = base::make_unique<IdleTask>(heap->isolate(), this);
+ V8::GetCurrentPlatform()->GetForegroundTaskRunner(isolate)->PostIdleTask(
+ std::move(task));
}
}
}
diff --git a/deps/v8/src/heap/scavenger-inl.h b/deps/v8/src/heap/scavenger-inl.h
index 649292085a..376b5e75aa 100644
--- a/deps/v8/src/heap/scavenger-inl.h
+++ b/deps/v8/src/heap/scavenger-inl.h
@@ -7,6 +7,7 @@
#include "src/heap/scavenger.h"
+#include "src/heap/incremental-marking-inl.h"
#include "src/heap/local-allocator-inl.h"
#include "src/objects-inl.h"
#include "src/objects/map.h"
@@ -14,6 +15,81 @@
namespace v8 {
namespace internal {
+void Scavenger::PromotionList::View::PushRegularObject(HeapObject* object,
+ int size) {
+ promotion_list_->PushRegularObject(task_id_, object, size);
+}
+
+void Scavenger::PromotionList::View::PushLargeObject(HeapObject* object,
+ Map* map, int size) {
+ promotion_list_->PushLargeObject(task_id_, object, map, size);
+}
+
+bool Scavenger::PromotionList::View::IsEmpty() {
+ return promotion_list_->IsEmpty();
+}
+
+size_t Scavenger::PromotionList::View::LocalPushSegmentSize() {
+ return promotion_list_->LocalPushSegmentSize(task_id_);
+}
+
+bool Scavenger::PromotionList::View::Pop(struct PromotionListEntry* entry) {
+ return promotion_list_->Pop(task_id_, entry);
+}
+
+bool Scavenger::PromotionList::View::IsGlobalPoolEmpty() {
+ return promotion_list_->IsGlobalPoolEmpty();
+}
+
+bool Scavenger::PromotionList::View::ShouldEagerlyProcessPromotionList() {
+ return promotion_list_->ShouldEagerlyProcessPromotionList(task_id_);
+}
+
+void Scavenger::PromotionList::PushRegularObject(int task_id,
+ HeapObject* object, int size) {
+ regular_object_promotion_list_.Push(task_id, ObjectAndSize(object, size));
+}
+
+void Scavenger::PromotionList::PushLargeObject(int task_id, HeapObject* object,
+ Map* map, int size) {
+ large_object_promotion_list_.Push(task_id, {object, map, size});
+}
+
+bool Scavenger::PromotionList::IsEmpty() {
+ return regular_object_promotion_list_.IsEmpty() &&
+ large_object_promotion_list_.IsEmpty();
+}
+
+size_t Scavenger::PromotionList::LocalPushSegmentSize(int task_id) {
+ return regular_object_promotion_list_.LocalPushSegmentSize(task_id) +
+ large_object_promotion_list_.LocalPushSegmentSize(task_id);
+}
+
+bool Scavenger::PromotionList::Pop(int task_id,
+ struct PromotionListEntry* entry) {
+ ObjectAndSize regular_object;
+ if (regular_object_promotion_list_.Pop(task_id, &regular_object)) {
+ entry->heap_object = regular_object.first;
+ entry->size = regular_object.second;
+ entry->map = entry->heap_object->map();
+ return true;
+ }
+ return large_object_promotion_list_.Pop(task_id, entry);
+}
+
+bool Scavenger::PromotionList::IsGlobalPoolEmpty() {
+ return regular_object_promotion_list_.IsGlobalPoolEmpty() &&
+ large_object_promotion_list_.IsGlobalPoolEmpty();
+}
+
+bool Scavenger::PromotionList::ShouldEagerlyProcessPromotionList(int task_id) {
+ // Threshold when to prioritize processing of the promotion list. Right
+ // now we only look into the regular object list.
+ const int kProcessPromotionListThreshold =
+ kRegularObjectPromotionListSegmentSize / 2;
+ return LocalPushSegmentSize(task_id) < kProcessPromotionListThreshold;
+}
+
// White list for objects that for sure only contain data.
bool Scavenger::ContainsOnlyData(VisitorId visitor_id) {
switch (visitor_id) {
@@ -38,7 +114,7 @@ void Scavenger::PageMemoryFence(MaybeObject* object) {
// Perform a dummy acquire load to tell TSAN that there is no data race
// with page initialization.
HeapObject* heap_object;
- if (object->ToStrongOrWeakHeapObject(&heap_object)) {
+ if (object->GetHeapObject(&heap_object)) {
MemoryChunk* chunk = MemoryChunk::FromAddress(heap_object->address());
CHECK_NOT_NULL(chunk->synchronized_heap());
}
@@ -71,8 +147,10 @@ bool Scavenger::MigrateObject(Map* map, HeapObject* source, HeapObject* target,
return true;
}
-bool Scavenger::SemiSpaceCopyObject(Map* map, HeapObjectReference** slot,
- HeapObject* object, int object_size) {
+CopyAndForwardResult Scavenger::SemiSpaceCopyObject(Map* map,
+ HeapObjectReference** slot,
+ HeapObject* object,
+ int object_size) {
DCHECK(heap()->AllowedToBeMigrated(object, NEW_SPACE));
AllocationAlignment alignment = HeapObject::RequiredAlignment(map);
AllocationResult allocation =
@@ -85,21 +163,26 @@ bool Scavenger::SemiSpaceCopyObject(Map* map, HeapObjectReference** slot,
const bool self_success = MigrateObject(map, object, target, object_size);
if (!self_success) {
allocator_.FreeLast(NEW_SPACE, target, object_size);
- MapWord map_word = object->map_word();
+ MapWord map_word = object->synchronized_map_word();
HeapObjectReference::Update(slot, map_word.ToForwardingAddress());
- return true;
+ DCHECK(!Heap::InFromSpace(*slot));
+ return Heap::InToSpace(*slot)
+ ? CopyAndForwardResult::SUCCESS_YOUNG_GENERATION
+ : CopyAndForwardResult::SUCCESS_OLD_GENERATION;
}
HeapObjectReference::Update(slot, target);
copied_list_.Push(ObjectAndSize(target, object_size));
copied_size_ += object_size;
- return true;
+ return CopyAndForwardResult::SUCCESS_YOUNG_GENERATION;
}
- return false;
+ return CopyAndForwardResult::FAILURE;
}
-bool Scavenger::PromoteObject(Map* map, HeapObjectReference** slot,
- HeapObject* object, int object_size) {
+CopyAndForwardResult Scavenger::PromoteObject(Map* map,
+ HeapObjectReference** slot,
+ HeapObject* object,
+ int object_size) {
AllocationAlignment alignment = HeapObject::RequiredAlignment(map);
AllocationResult allocation =
allocator_.Allocate(OLD_SPACE, object_size, alignment);
@@ -111,61 +194,112 @@ bool Scavenger::PromoteObject(Map* map, HeapObjectReference** slot,
const bool self_success = MigrateObject(map, object, target, object_size);
if (!self_success) {
allocator_.FreeLast(OLD_SPACE, target, object_size);
- MapWord map_word = object->map_word();
+ MapWord map_word = object->synchronized_map_word();
HeapObjectReference::Update(slot, map_word.ToForwardingAddress());
- return true;
+ DCHECK(!Heap::InFromSpace(*slot));
+ return Heap::InToSpace(*slot)
+ ? CopyAndForwardResult::SUCCESS_YOUNG_GENERATION
+ : CopyAndForwardResult::SUCCESS_OLD_GENERATION;
}
HeapObjectReference::Update(slot, target);
if (!ContainsOnlyData(map->visitor_id())) {
- promotion_list_.Push(ObjectAndSize(target, object_size));
+ promotion_list_.PushRegularObject(target, object_size);
}
promoted_size_ += object_size;
+ return CopyAndForwardResult::SUCCESS_OLD_GENERATION;
+ }
+ return CopyAndForwardResult::FAILURE;
+}
+
+SlotCallbackResult Scavenger::RememberedSetEntryNeeded(
+ CopyAndForwardResult result) {
+ DCHECK_NE(CopyAndForwardResult::FAILURE, result);
+ return result == CopyAndForwardResult::SUCCESS_YOUNG_GENERATION ? KEEP_SLOT
+ : REMOVE_SLOT;
+}
+
+bool Scavenger::HandleLargeObject(Map* map, HeapObject* object,
+ int object_size) {
+ if (V8_UNLIKELY(FLAG_young_generation_large_objects &&
+ object_size > kMaxNewSpaceHeapObjectSize)) {
+ DCHECK_EQ(NEW_LO_SPACE,
+ MemoryChunk::FromHeapObject(object)->owner()->identity());
+ if (base::AsAtomicPointer::Release_CompareAndSwap(
+ reinterpret_cast<HeapObject**>(object->address()), map,
+ MapWord::FromForwardingAddress(object).ToMap()) == map) {
+ surviving_new_large_objects_.insert({object, map});
+
+ if (!ContainsOnlyData(map->visitor_id())) {
+ promotion_list_.PushLargeObject(object, map, object_size);
+ }
+ }
return true;
}
return false;
}
-void Scavenger::EvacuateObjectDefault(Map* map, HeapObjectReference** slot,
- HeapObject* object, int object_size) {
+SlotCallbackResult Scavenger::EvacuateObjectDefault(Map* map,
+ HeapObjectReference** slot,
+ HeapObject* object,
+ int object_size) {
SLOW_DCHECK(object_size <= Page::kAllocatableMemory);
SLOW_DCHECK(object->SizeFromMap(map) == object_size);
+ CopyAndForwardResult result;
+
+ if (HandleLargeObject(map, object, object_size)) {
+ return REMOVE_SLOT;
+ }
if (!heap()->ShouldBePromoted(object->address())) {
// A semi-space copy may fail due to fragmentation. In that case, we
// try to promote the object.
- if (SemiSpaceCopyObject(map, slot, object, object_size)) return;
+ result = SemiSpaceCopyObject(map, slot, object, object_size);
+ if (result != CopyAndForwardResult::FAILURE) {
+ return RememberedSetEntryNeeded(result);
+ }
}
- if (PromoteObject(map, slot, object, object_size)) return;
+ // We may want to promote this object if the object was already semi-space
+ // copied in a previes young generation GC or if the semi-space copy above
+ // failed.
+ result = PromoteObject(map, slot, object, object_size);
+ if (result != CopyAndForwardResult::FAILURE) {
+ return RememberedSetEntryNeeded(result);
+ }
- // If promotion failed, we try to copy the object to the other semi-space
- if (SemiSpaceCopyObject(map, slot, object, object_size)) return;
+ // If promotion failed, we try to copy the object to the other semi-space.
+ result = SemiSpaceCopyObject(map, slot, object, object_size);
+ if (result != CopyAndForwardResult::FAILURE) {
+ return RememberedSetEntryNeeded(result);
+ }
heap()->FatalProcessOutOfMemory("Scavenger: semi-space copy");
+ UNREACHABLE();
}
-void Scavenger::EvacuateThinString(Map* map, HeapObject** slot,
- ThinString* object, int object_size) {
+SlotCallbackResult Scavenger::EvacuateThinString(Map* map, HeapObject** slot,
+ ThinString* object,
+ int object_size) {
if (!is_incremental_marking_) {
- // Loading actual is fine in a parallel setting is there is no write.
+ // The ThinString should die after Scavenge, so avoid writing the proper
+ // forwarding pointer and instead just signal the actual object as forwarded
+ // reference.
String* actual = object->actual();
- object->set_length(0);
- *slot = actual;
- // ThinStrings always refer to internalized strings, which are
- // always in old space.
+ // ThinStrings always refer to internalized strings, which are always in old
+ // space.
DCHECK(!Heap::InNewSpace(actual));
- base::AsAtomicPointer::Relaxed_Store(
- reinterpret_cast<Map**>(object->address()),
- MapWord::FromForwardingAddress(actual).ToMap());
- return;
+ *slot = actual;
+ return REMOVE_SLOT;
}
- EvacuateObjectDefault(map, reinterpret_cast<HeapObjectReference**>(slot),
- object, object_size);
+ return EvacuateObjectDefault(
+ map, reinterpret_cast<HeapObjectReference**>(slot), object, object_size);
}
-void Scavenger::EvacuateShortcutCandidate(Map* map, HeapObject** slot,
- ConsString* object, int object_size) {
+SlotCallbackResult Scavenger::EvacuateShortcutCandidate(Map* map,
+ HeapObject** slot,
+ ConsString* object,
+ int object_size) {
DCHECK(IsShortcutCandidate(map->instance_type()));
if (!is_incremental_marking_ &&
object->unchecked_second() == ReadOnlyRoots(heap()).empty_string()) {
@@ -174,37 +308,38 @@ void Scavenger::EvacuateShortcutCandidate(Map* map, HeapObject** slot,
*slot = first;
if (!Heap::InNewSpace(first)) {
- base::AsAtomicPointer::Relaxed_Store(
+ base::AsAtomicPointer::Release_Store(
reinterpret_cast<Map**>(object->address()),
MapWord::FromForwardingAddress(first).ToMap());
- return;
+ return REMOVE_SLOT;
}
- MapWord first_word = first->map_word();
+ MapWord first_word = first->synchronized_map_word();
if (first_word.IsForwardingAddress()) {
HeapObject* target = first_word.ToForwardingAddress();
*slot = target;
- base::AsAtomicPointer::Relaxed_Store(
+ base::AsAtomicPointer::Release_Store(
reinterpret_cast<Map**>(object->address()),
MapWord::FromForwardingAddress(target).ToMap());
- return;
+ return Heap::InToSpace(target) ? KEEP_SLOT : REMOVE_SLOT;
}
Map* map = first_word.ToMap();
- EvacuateObjectDefault(map, reinterpret_cast<HeapObjectReference**>(slot),
- first, first->SizeFromMap(map));
- base::AsAtomicPointer::Relaxed_Store(
+ SlotCallbackResult result = EvacuateObjectDefault(
+ map, reinterpret_cast<HeapObjectReference**>(slot), first,
+ first->SizeFromMap(map));
+ base::AsAtomicPointer::Release_Store(
reinterpret_cast<Map**>(object->address()),
MapWord::FromForwardingAddress(*slot).ToMap());
- return;
+ return result;
}
- EvacuateObjectDefault(map, reinterpret_cast<HeapObjectReference**>(slot),
- object, object_size);
+ return EvacuateObjectDefault(
+ map, reinterpret_cast<HeapObjectReference**>(slot), object, object_size);
}
-void Scavenger::EvacuateObject(HeapObjectReference** slot, Map* map,
- HeapObject* source) {
+SlotCallbackResult Scavenger::EvacuateObject(HeapObjectReference** slot,
+ Map* map, HeapObject* source) {
SLOW_DCHECK(Heap::InFromSpace(source));
SLOW_DCHECK(!MapWord::FromMap(map).IsForwardingAddress());
int size = source->SizeFromMap(map);
@@ -213,23 +348,22 @@ void Scavenger::EvacuateObject(HeapObjectReference** slot, Map* map,
switch (map->visitor_id()) {
case kVisitThinString:
// At the moment we don't allow weak pointers to thin strings.
- DCHECK(!(*slot)->IsWeakHeapObject());
- EvacuateThinString(map, reinterpret_cast<HeapObject**>(slot),
- reinterpret_cast<ThinString*>(source), size);
- break;
+ DCHECK(!(*slot)->IsWeak());
+ return EvacuateThinString(map, reinterpret_cast<HeapObject**>(slot),
+ reinterpret_cast<ThinString*>(source), size);
case kVisitShortcutCandidate:
- DCHECK(!(*slot)->IsWeakHeapObject());
+ DCHECK(!(*slot)->IsWeak());
// At the moment we don't allow weak pointers to cons strings.
- EvacuateShortcutCandidate(map, reinterpret_cast<HeapObject**>(slot),
- reinterpret_cast<ConsString*>(source), size);
- break;
+ return EvacuateShortcutCandidate(
+ map, reinterpret_cast<HeapObject**>(slot),
+ reinterpret_cast<ConsString*>(source), size);
default:
- EvacuateObjectDefault(map, slot, source, size);
- break;
+ return EvacuateObjectDefault(map, slot, source, size);
}
}
-void Scavenger::ScavengeObject(HeapObjectReference** p, HeapObject* object) {
+SlotCallbackResult Scavenger::ScavengeObject(HeapObjectReference** p,
+ HeapObject* object) {
DCHECK(Heap::InFromSpace(object));
// Synchronized load that consumes the publishing CAS of MigrateObject.
@@ -240,20 +374,21 @@ void Scavenger::ScavengeObject(HeapObjectReference** p, HeapObject* object) {
if (first_word.IsForwardingAddress()) {
HeapObject* dest = first_word.ToForwardingAddress();
DCHECK(Heap::InFromSpace(*p));
- if ((*p)->IsWeakHeapObject()) {
+ if ((*p)->IsWeak()) {
*p = HeapObjectReference::Weak(dest);
} else {
- DCHECK((*p)->IsStrongHeapObject());
+ DCHECK((*p)->IsStrong());
*p = HeapObjectReference::Strong(dest);
}
- return;
+ DCHECK(Heap::InToSpace(dest) || !Heap::InNewSpace((dest)));
+ return Heap::InToSpace(dest) ? KEEP_SLOT : REMOVE_SLOT;
}
Map* map = first_word.ToMap();
// AllocationMementos are unrooted and shouldn't survive a scavenge
DCHECK_NE(ReadOnlyRoots(heap()).allocation_memento_map(), map);
// Call the slow part of scavenge object.
- EvacuateObject(p, map, object);
+ return EvacuateObject(p, map, object);
}
SlotCallbackResult Scavenger::CheckAndScavengeObject(Heap* heap,
@@ -261,23 +396,13 @@ SlotCallbackResult Scavenger::CheckAndScavengeObject(Heap* heap,
MaybeObject** slot = reinterpret_cast<MaybeObject**>(slot_address);
MaybeObject* object = *slot;
if (Heap::InFromSpace(object)) {
- HeapObject* heap_object;
- bool success = object->ToStrongOrWeakHeapObject(&heap_object);
- USE(success);
- DCHECK(success);
+ HeapObject* heap_object = object->GetHeapObject();
DCHECK(heap_object->IsHeapObject());
- ScavengeObject(reinterpret_cast<HeapObjectReference**>(slot), heap_object);
-
- object = *slot;
- // If the object was in from space before and is after executing the
- // callback in to space, the object is still live.
- // Unfortunately, we do not know about the slot. It could be in a
- // just freed free space object.
- PageMemoryFence(object);
- if (Heap::InToSpace(object)) {
- return KEEP_SLOT;
- }
+ SlotCallbackResult result = ScavengeObject(
+ reinterpret_cast<HeapObjectReference**>(slot), heap_object);
+ DCHECK_IMPLIES(result == REMOVE_SLOT, !Heap::InNewSpace(*slot));
+ return result;
} else if (Heap::InToSpace(object)) {
// Already updated slot. This can happen when processing of the work list
// is interleaved with processing roots.
@@ -305,7 +430,7 @@ void ScavengeVisitor::VisitPointers(HeapObject* host, MaybeObject** start,
if (!Heap::InNewSpace(object)) continue;
// Treat the weak reference as strong.
HeapObject* heap_object;
- if (object->ToStrongOrWeakHeapObject(&heap_object)) {
+ if (object->GetHeapObject(&heap_object)) {
scavenger_->ScavengeObject(reinterpret_cast<HeapObjectReference**>(p),
heap_object);
} else {
diff --git a/deps/v8/src/heap/scavenger.cc b/deps/v8/src/heap/scavenger.cc
index f8c6d496ce..4c63ed099a 100644
--- a/deps/v8/src/heap/scavenger.cc
+++ b/deps/v8/src/heap/scavenger.cc
@@ -4,17 +4,72 @@
#include "src/heap/scavenger.h"
+#include "src/heap/array-buffer-collector.h"
#include "src/heap/barrier.h"
+#include "src/heap/gc-tracer.h"
#include "src/heap/heap-inl.h"
+#include "src/heap/item-parallel-job.h"
#include "src/heap/mark-compact-inl.h"
#include "src/heap/objects-visiting-inl.h"
#include "src/heap/scavenger-inl.h"
#include "src/heap/sweeper.h"
#include "src/objects-body-descriptors-inl.h"
+#include "src/utils-inl.h"
namespace v8 {
namespace internal {
+class PageScavengingItem final : public ItemParallelJob::Item {
+ public:
+ explicit PageScavengingItem(MemoryChunk* chunk) : chunk_(chunk) {}
+ ~PageScavengingItem() override = default;
+
+ void Process(Scavenger* scavenger) { scavenger->ScavengePage(chunk_); }
+
+ private:
+ MemoryChunk* const chunk_;
+};
+
+class ScavengingTask final : public ItemParallelJob::Task {
+ public:
+ ScavengingTask(Heap* heap, Scavenger* scavenger, OneshotBarrier* barrier)
+ : ItemParallelJob::Task(heap->isolate()),
+ heap_(heap),
+ scavenger_(scavenger),
+ barrier_(barrier) {}
+
+ void RunInParallel() final {
+ TRACE_BACKGROUND_GC(
+ heap_->tracer(),
+ GCTracer::BackgroundScope::SCAVENGER_BACKGROUND_SCAVENGE_PARALLEL);
+ double scavenging_time = 0.0;
+ {
+ barrier_->Start();
+ TimedScope scope(&scavenging_time);
+ PageScavengingItem* item = nullptr;
+ while ((item = GetItem<PageScavengingItem>()) != nullptr) {
+ item->Process(scavenger_);
+ item->MarkFinished();
+ }
+ do {
+ scavenger_->Process(barrier_);
+ } while (!barrier_->Wait());
+ scavenger_->Process();
+ }
+ if (FLAG_trace_parallel_scavenge) {
+ PrintIsolate(heap_->isolate(),
+ "scavenge[%p]: time=%.2f copied=%zu promoted=%zu\n",
+ static_cast<void*>(this), scavenging_time,
+ scavenger_->bytes_copied(), scavenger_->bytes_promoted());
+ }
+ };
+
+ private:
+ Heap* const heap_;
+ Scavenger* const scavenger_;
+ OneshotBarrier* const barrier_;
+};
+
class IterateAndScavengePromotedObjectsVisitor final : public ObjectVisitor {
public:
IterateAndScavengePromotedObjectsVisitor(Heap* heap, Scavenger* scavenger,
@@ -40,7 +95,7 @@ class IterateAndScavengePromotedObjectsVisitor final : public ObjectVisitor {
for (MaybeObject** slot = start; slot < end; ++slot) {
MaybeObject* target = *slot;
HeapObject* heap_object;
- if (target->ToStrongOrWeakHeapObject(&heap_object)) {
+ if (target->GetHeapObject(&heap_object)) {
HandleSlot(host, reinterpret_cast<Address>(slot), heap_object);
}
}
@@ -53,15 +108,13 @@ class IterateAndScavengePromotedObjectsVisitor final : public ObjectVisitor {
scavenger_->PageMemoryFence(reinterpret_cast<MaybeObject*>(target));
if (Heap::InFromSpace(target)) {
- scavenger_->ScavengeObject(slot, target);
- bool success = (*slot)->ToStrongOrWeakHeapObject(&target);
+ SlotCallbackResult result = scavenger_->ScavengeObject(slot, target);
+ bool success = (*slot)->GetHeapObject(&target);
USE(success);
DCHECK(success);
- scavenger_->PageMemoryFence(reinterpret_cast<MaybeObject*>(target));
- if (Heap::InNewSpace(target)) {
+ if (result == KEEP_SLOT) {
SLOW_DCHECK(target->IsHeapObject());
- SLOW_DCHECK(Heap::InToSpace(target));
RememberedSet<OLD_TO_NEW>::Insert(Page::FromAddress(slot_address),
slot_address);
}
@@ -79,9 +132,204 @@ class IterateAndScavengePromotedObjectsVisitor final : public ObjectVisitor {
const bool record_slots_;
};
-Scavenger::Scavenger(Heap* heap, bool is_logging, CopiedList* copied_list,
- PromotionList* promotion_list, int task_id)
- : heap_(heap),
+static bool IsUnscavengedHeapObject(Heap* heap, Object** p) {
+ return Heap::InFromSpace(*p) &&
+ !HeapObject::cast(*p)->map_word().IsForwardingAddress();
+}
+
+class ScavengeWeakObjectRetainer : public WeakObjectRetainer {
+ public:
+ Object* RetainAs(Object* object) override {
+ if (!Heap::InFromSpace(object)) {
+ return object;
+ }
+
+ MapWord map_word = HeapObject::cast(object)->map_word();
+ if (map_word.IsForwardingAddress()) {
+ return map_word.ToForwardingAddress();
+ }
+ return nullptr;
+ }
+};
+
+ScavengerCollector::ScavengerCollector(Heap* heap)
+ : isolate_(heap->isolate()), heap_(heap), parallel_scavenge_semaphore_(0) {}
+
+void ScavengerCollector::CollectGarbage() {
+ ItemParallelJob job(isolate_->cancelable_task_manager(),
+ &parallel_scavenge_semaphore_);
+ const int kMainThreadId = 0;
+ Scavenger* scavengers[kMaxScavengerTasks];
+ const bool is_logging = isolate_->LogObjectRelocation();
+ const int num_scavenge_tasks = NumberOfScavengeTasks();
+ OneshotBarrier barrier;
+ Scavenger::CopiedList copied_list(num_scavenge_tasks);
+ Scavenger::PromotionList promotion_list(num_scavenge_tasks);
+ for (int i = 0; i < num_scavenge_tasks; i++) {
+ scavengers[i] = new Scavenger(this, heap_, is_logging, &copied_list,
+ &promotion_list, i);
+ job.AddTask(new ScavengingTask(heap_, scavengers[i], &barrier));
+ }
+
+ {
+ Sweeper* sweeper = heap_->mark_compact_collector()->sweeper();
+ // Pause the concurrent sweeper.
+ Sweeper::PauseOrCompleteScope pause_scope(sweeper);
+ // Filter out pages from the sweeper that need to be processed for old to
+ // new slots by the Scavenger. After processing, the Scavenger adds back
+ // pages that are still unsweeped. This way the Scavenger has exclusive
+ // access to the slots of a page and can completely avoid any locks on
+ // the page itself.
+ Sweeper::FilterSweepingPagesScope filter_scope(sweeper, pause_scope);
+ filter_scope.FilterOldSpaceSweepingPages(
+ [](Page* page) { return !page->ContainsSlots<OLD_TO_NEW>(); });
+ RememberedSet<OLD_TO_NEW>::IterateMemoryChunks(
+ heap_, [&job](MemoryChunk* chunk) {
+ job.AddItem(new PageScavengingItem(chunk));
+ });
+
+ RootScavengeVisitor root_scavenge_visitor(scavengers[kMainThreadId]);
+
+ {
+ // Identify weak unmodified handles. Requires an unmodified graph.
+ TRACE_GC(
+ heap_->tracer(),
+ GCTracer::Scope::SCAVENGER_SCAVENGE_WEAK_GLOBAL_HANDLES_IDENTIFY);
+ isolate_->global_handles()->IdentifyWeakUnmodifiedObjects(
+ &JSObject::IsUnmodifiedApiObject);
+ }
+ {
+ // Copy roots.
+ TRACE_GC(heap_->tracer(), GCTracer::Scope::SCAVENGER_SCAVENGE_ROOTS);
+ heap_->IterateRoots(&root_scavenge_visitor, VISIT_ALL_IN_SCAVENGE);
+ }
+ {
+ // Parallel phase scavenging all copied and promoted objects.
+ TRACE_GC(heap_->tracer(), GCTracer::Scope::SCAVENGER_SCAVENGE_PARALLEL);
+ job.Run(isolate_->async_counters());
+ DCHECK(copied_list.IsEmpty());
+ DCHECK(promotion_list.IsEmpty());
+ }
+ {
+ // Scavenge weak global handles.
+ TRACE_GC(heap_->tracer(),
+ GCTracer::Scope::SCAVENGER_SCAVENGE_WEAK_GLOBAL_HANDLES_PROCESS);
+ isolate_->global_handles()->MarkNewSpaceWeakUnmodifiedObjectsPending(
+ &IsUnscavengedHeapObject);
+ isolate_->global_handles()
+ ->IterateNewSpaceWeakUnmodifiedRootsForFinalizers(
+ &root_scavenge_visitor);
+ scavengers[kMainThreadId]->Process();
+
+ DCHECK(copied_list.IsEmpty());
+ DCHECK(promotion_list.IsEmpty());
+ isolate_->global_handles()
+ ->IterateNewSpaceWeakUnmodifiedRootsForPhantomHandles(
+ &root_scavenge_visitor, &IsUnscavengedHeapObject);
+ }
+
+ {
+ // Finalize parallel scavenging.
+ TRACE_GC(heap_->tracer(), GCTracer::Scope::SCAVENGER_SCAVENGE_FINALIZE);
+
+ for (int i = 0; i < num_scavenge_tasks; i++) {
+ scavengers[i]->Finalize();
+ delete scavengers[i];
+ }
+
+ HandleSurvivingNewLargeObjects();
+ }
+ }
+
+ {
+ // Update references into new space
+ TRACE_GC(heap_->tracer(), GCTracer::Scope::SCAVENGER_SCAVENGE_UPDATE_REFS);
+ heap_->UpdateNewSpaceReferencesInExternalStringTable(
+ &Heap::UpdateNewSpaceReferenceInExternalStringTableEntry);
+
+ heap_->incremental_marking()->UpdateMarkingWorklistAfterScavenge();
+ }
+
+ if (FLAG_concurrent_marking) {
+ // Ensure that concurrent marker does not track pages that are
+ // going to be unmapped.
+ for (Page* p :
+ PageRange(heap_->new_space()->from_space().first_page(), nullptr)) {
+ heap_->concurrent_marking()->ClearLiveness(p);
+ }
+ }
+
+ ScavengeWeakObjectRetainer weak_object_retainer;
+ heap_->ProcessYoungWeakReferences(&weak_object_retainer);
+
+ // Set age mark.
+ heap_->new_space_->set_age_mark(heap_->new_space()->top());
+
+ {
+ TRACE_GC(heap_->tracer(), GCTracer::Scope::SCAVENGER_PROCESS_ARRAY_BUFFERS);
+ ArrayBufferTracker::PrepareToFreeDeadInNewSpace(heap_);
+ }
+ heap_->array_buffer_collector()->FreeAllocations();
+
+ RememberedSet<OLD_TO_NEW>::IterateMemoryChunks(heap_, [](MemoryChunk* chunk) {
+ if (chunk->SweepingDone()) {
+ RememberedSet<OLD_TO_NEW>::FreeEmptyBuckets(chunk);
+ } else {
+ RememberedSet<OLD_TO_NEW>::PreFreeEmptyBuckets(chunk);
+ }
+ });
+
+ // Update how much has survived scavenge.
+ heap_->IncrementYoungSurvivorsCounter(heap_->SurvivedNewSpaceObjectSize());
+
+ // Scavenger may find new wrappers by iterating objects promoted onto a black
+ // page.
+ heap_->local_embedder_heap_tracer()->RegisterWrappersWithRemoteTracer();
+}
+
+void ScavengerCollector::HandleSurvivingNewLargeObjects() {
+ for (SurvivingNewLargeObjectMapEntry update_info :
+ surviving_new_large_objects_) {
+ HeapObject* object = update_info.first;
+ Map* map = update_info.second;
+ // Order is important here. We have to re-install the map to have access
+ // to meta-data like size during page promotion.
+ object->set_map_word(MapWord::FromMap(map));
+ LargePage* page = LargePage::FromHeapObject(object);
+ heap_->lo_space()->PromoteNewLargeObject(page);
+ }
+ DCHECK(heap_->new_lo_space()->IsEmpty());
+}
+
+void ScavengerCollector::MergeSurvivingNewLargeObjects(
+ const SurvivingNewLargeObjectsMap& objects) {
+ for (SurvivingNewLargeObjectMapEntry object : objects) {
+ bool success = surviving_new_large_objects_.insert(object).second;
+ USE(success);
+ DCHECK(success);
+ }
+}
+
+int ScavengerCollector::NumberOfScavengeTasks() {
+ if (!FLAG_parallel_scavenge) return 1;
+ const int num_scavenge_tasks =
+ static_cast<int>(heap_->new_space()->TotalCapacity()) / MB;
+ static int num_cores = V8::GetCurrentPlatform()->NumberOfWorkerThreads() + 1;
+ int tasks =
+ Max(1, Min(Min(num_scavenge_tasks, kMaxScavengerTasks), num_cores));
+ if (!heap_->CanExpandOldGeneration(
+ static_cast<size_t>(tasks * Page::kPageSize))) {
+ // Optimize for memory usage near the heap limit.
+ tasks = 1;
+ }
+ return tasks;
+}
+
+Scavenger::Scavenger(ScavengerCollector* collector, Heap* heap, bool is_logging,
+ CopiedList* copied_list, PromotionList* promotion_list,
+ int task_id)
+ : collector_(collector),
+ heap_(heap),
promotion_list_(promotion_list, task_id),
copied_list_(copied_list, task_id),
local_pretenuring_feedback_(kInitialLocalPretenuringFeedbackCapacity),
@@ -92,7 +340,8 @@ Scavenger::Scavenger(Heap* heap, bool is_logging, CopiedList* copied_list,
is_incremental_marking_(heap->incremental_marking()->IsMarking()),
is_compacting_(heap->incremental_marking()->IsCompacting()) {}
-void Scavenger::IterateAndScavengePromotedObject(HeapObject* target, int size) {
+void Scavenger::IterateAndScavengePromotedObject(HeapObject* target, Map* map,
+ int size) {
// We are not collecting slots on new space objects during mutation thus we
// have to scan for pointers to evacuation candidates when we promote
// objects. But we should not record any slots in non-black objects. Grey
@@ -103,7 +352,7 @@ void Scavenger::IterateAndScavengePromotedObject(HeapObject* target, int size) {
is_compacting_ &&
heap()->incremental_marking()->atomic_marking_state()->IsBlack(target);
IterateAndScavengePromotedObjectsVisitor visitor(heap(), this, record_slots);
- target->IterateBodyFast(target->map(), size, &visitor);
+ target->IterateBodyFast(map, size, &visitor);
}
void Scavenger::AddPageToSweeperIfNecessary(MemoryChunk* page) {
@@ -136,9 +385,6 @@ void Scavenger::ScavengePage(MemoryChunk* page) {
void Scavenger::Process(OneshotBarrier* barrier) {
TRACE_EVENT0(TRACE_DISABLED_BY_DEFAULT("v8.gc"), "Scavenger::Process");
- // Threshold when to switch processing the promotion list to avoid
- // allocating too much backing store in the worklist.
- const int kProcessPromotionListThreshold = kPromotionListSegmentSize / 2;
ScavengeVisitor scavenge_visitor(this);
const bool have_barrier = barrier != nullptr;
@@ -147,8 +393,7 @@ void Scavenger::Process(OneshotBarrier* barrier) {
do {
done = true;
ObjectAndSize object_and_size;
- while ((promotion_list_.LocalPushSegmentSize() <
- kProcessPromotionListThreshold) &&
+ while (promotion_list_.ShouldEagerlyProcessPromotionList() &&
copied_list_.Pop(&object_and_size)) {
scavenge_visitor.Visit(object_and_size.first);
done = false;
@@ -159,11 +404,11 @@ void Scavenger::Process(OneshotBarrier* barrier) {
}
}
- while (promotion_list_.Pop(&object_and_size)) {
- HeapObject* target = object_and_size.first;
- int size = object_and_size.second;
+ struct PromotionListEntry entry;
+ while (promotion_list_.Pop(&entry)) {
+ HeapObject* target = entry.heap_object;
DCHECK(!target->IsMap());
- IterateAndScavengePromotedObject(target, size);
+ IterateAndScavengePromotedObject(target, entry.map, entry.size);
done = false;
if (have_barrier && ((++objects % kInterruptThreshold) == 0)) {
if (!promotion_list_.IsGlobalPoolEmpty()) {
@@ -178,6 +423,7 @@ void Scavenger::Finalize() {
heap()->MergeAllocationSitePretenuringFeedback(local_pretenuring_feedback_);
heap()->IncrementSemiSpaceCopiedObjectSize(copied_size_);
heap()->IncrementPromotedObjectsSize(promoted_size_);
+ collector_->MergeSurvivingNewLargeObjects(surviving_new_large_objects_);
allocator_.Finalize();
}
diff --git a/deps/v8/src/heap/scavenger.h b/deps/v8/src/heap/scavenger.h
index 4e6753f6ce..b984102c6b 100644
--- a/deps/v8/src/heap/scavenger.h
+++ b/deps/v8/src/heap/scavenger.h
@@ -16,17 +16,101 @@ namespace internal {
class OneshotBarrier;
+enum class CopyAndForwardResult {
+ SUCCESS_YOUNG_GENERATION,
+ SUCCESS_OLD_GENERATION,
+ FAILURE
+};
+
+using ObjectAndSize = std::pair<HeapObject*, int>;
+using SurvivingNewLargeObjectsMap = std::unordered_map<HeapObject*, Map*>;
+using SurvivingNewLargeObjectMapEntry = std::pair<HeapObject*, Map*>;
+
+class ScavengerCollector {
+ public:
+ static const int kMaxScavengerTasks = 8;
+
+ explicit ScavengerCollector(Heap* heap);
+
+ void CollectGarbage();
+
+ private:
+ void MergeSurvivingNewLargeObjects(
+ const SurvivingNewLargeObjectsMap& objects);
+
+ int NumberOfScavengeTasks();
+
+ void HandleSurvivingNewLargeObjects();
+
+ Isolate* const isolate_;
+ Heap* const heap_;
+ base::Semaphore parallel_scavenge_semaphore_;
+ SurvivingNewLargeObjectsMap surviving_new_large_objects_;
+
+ friend class Scavenger;
+};
+
class Scavenger {
public:
+ struct PromotionListEntry {
+ HeapObject* heap_object;
+ Map* map;
+ int size;
+ };
+
+ class PromotionList {
+ public:
+ class View {
+ public:
+ View(PromotionList* promotion_list, int task_id)
+ : promotion_list_(promotion_list), task_id_(task_id) {}
+
+ inline void PushRegularObject(HeapObject* object, int size);
+ inline void PushLargeObject(HeapObject* object, Map* map, int size);
+ inline bool IsEmpty();
+ inline size_t LocalPushSegmentSize();
+ inline bool Pop(struct PromotionListEntry* entry);
+ inline bool IsGlobalPoolEmpty();
+ inline bool ShouldEagerlyProcessPromotionList();
+
+ private:
+ PromotionList* promotion_list_;
+ int task_id_;
+ };
+
+ explicit PromotionList(int num_tasks)
+ : regular_object_promotion_list_(num_tasks),
+ large_object_promotion_list_(num_tasks) {}
+
+ inline void PushRegularObject(int task_id, HeapObject* object, int size);
+ inline void PushLargeObject(int task_id, HeapObject* object, Map* map,
+ int size);
+ inline bool IsEmpty();
+ inline size_t LocalPushSegmentSize(int task_id);
+ inline bool Pop(int task_id, struct PromotionListEntry* entry);
+ inline bool IsGlobalPoolEmpty();
+ inline bool ShouldEagerlyProcessPromotionList(int task_id);
+
+ private:
+ static const int kRegularObjectPromotionListSegmentSize = 256;
+ static const int kLargeObjectPromotionListSegmentSize = 4;
+
+ using RegularObjectPromotionList =
+ Worklist<ObjectAndSize, kRegularObjectPromotionListSegmentSize>;
+ using LargeObjectPromotionList =
+ Worklist<PromotionListEntry, kLargeObjectPromotionListSegmentSize>;
+
+ RegularObjectPromotionList regular_object_promotion_list_;
+ LargeObjectPromotionList large_object_promotion_list_;
+ };
+
static const int kCopiedListSegmentSize = 256;
- static const int kPromotionListSegmentSize = 256;
- using ObjectAndSize = std::pair<HeapObject*, int>;
using CopiedList = Worklist<ObjectAndSize, kCopiedListSegmentSize>;
- using PromotionList = Worklist<ObjectAndSize, kPromotionListSegmentSize>;
- Scavenger(Heap* heap, bool is_logging, CopiedList* copied_list,
- PromotionList* promotion_list, int task_id);
+ Scavenger(ScavengerCollector* collector, Heap* heap, bool is_logging,
+ CopiedList* copied_list, PromotionList* promotion_list,
+ int task_id);
// Entry point for scavenging an old generation page. For scavenging single
// objects see RootScavengingVisitor and ScavengeVisitor below.
@@ -61,39 +145,52 @@ class Scavenger {
// Scavenges an object |object| referenced from slot |p|. |object| is required
// to be in from space.
- inline void ScavengeObject(HeapObjectReference** p, HeapObject* object);
+ inline SlotCallbackResult ScavengeObject(HeapObjectReference** p,
+ HeapObject* object);
// Copies |source| to |target| and sets the forwarding pointer in |source|.
V8_INLINE bool MigrateObject(Map* map, HeapObject* source, HeapObject* target,
int size);
- V8_INLINE bool SemiSpaceCopyObject(Map* map, HeapObjectReference** slot,
- HeapObject* object, int object_size);
+ V8_INLINE SlotCallbackResult
+ RememberedSetEntryNeeded(CopyAndForwardResult result);
- V8_INLINE bool PromoteObject(Map* map, HeapObjectReference** slot,
- HeapObject* object, int object_size);
+ V8_INLINE CopyAndForwardResult SemiSpaceCopyObject(Map* map,
+ HeapObjectReference** slot,
+ HeapObject* object,
+ int object_size);
- V8_INLINE void EvacuateObject(HeapObjectReference** slot, Map* map,
- HeapObject* source);
+ V8_INLINE CopyAndForwardResult PromoteObject(Map* map,
+ HeapObjectReference** slot,
+ HeapObject* object,
+ int object_size);
- // Different cases for object evacuation.
+ V8_INLINE SlotCallbackResult EvacuateObject(HeapObjectReference** slot,
+ Map* map, HeapObject* source);
- V8_INLINE void EvacuateObjectDefault(Map* map, HeapObjectReference** slot,
- HeapObject* object, int object_size);
+ V8_INLINE bool HandleLargeObject(Map* map, HeapObject* object,
+ int object_size);
- V8_INLINE void EvacuateJSFunction(Map* map, HeapObject** slot,
- JSFunction* object, int object_size);
+ // Different cases for object evacuation.
+ V8_INLINE SlotCallbackResult EvacuateObjectDefault(Map* map,
+ HeapObjectReference** slot,
+ HeapObject* object,
+ int object_size);
- inline void EvacuateThinString(Map* map, HeapObject** slot,
- ThinString* object, int object_size);
+ inline SlotCallbackResult EvacuateThinString(Map* map, HeapObject** slot,
+ ThinString* object,
+ int object_size);
- inline void EvacuateShortcutCandidate(Map* map, HeapObject** slot,
- ConsString* object, int object_size);
+ inline SlotCallbackResult EvacuateShortcutCandidate(Map* map,
+ HeapObject** slot,
+ ConsString* object,
+ int object_size);
- void IterateAndScavengePromotedObject(HeapObject* target, int size);
+ void IterateAndScavengePromotedObject(HeapObject* target, Map* map, int size);
static inline bool ContainsOnlyData(VisitorId visitor_id);
+ ScavengerCollector* const collector_;
Heap* const heap_;
PromotionList::View promotion_list_;
CopiedList::View copied_list_;
@@ -101,6 +198,7 @@ class Scavenger {
size_t copied_size_;
size_t promoted_size_;
LocalAllocator allocator_;
+ SurvivingNewLargeObjectsMap surviving_new_large_objects_;
const bool is_logging_;
const bool is_incremental_marking_;
const bool is_compacting_;
diff --git a/deps/v8/src/heap/setup-heap-internal.cc b/deps/v8/src/heap/setup-heap-internal.cc
index 2742cd9c9d..5790b82907 100644
--- a/deps/v8/src/heap/setup-heap-internal.cc
+++ b/deps/v8/src/heap/setup-heap-internal.cc
@@ -23,11 +23,13 @@
#include "src/objects/dictionary.h"
#include "src/objects/literal-objects-inl.h"
#include "src/objects/map.h"
+#include "src/objects/microtask-queue.h"
#include "src/objects/microtask.h"
#include "src/objects/module.h"
#include "src/objects/promise.h"
#include "src/objects/script.h"
#include "src/objects/shared-function-info.h"
+#include "src/objects/stack-frame-info.h"
#include "src/objects/string.h"
#include "src/regexp/jsregexp.h"
#include "src/wasm/wasm-objects.h"
@@ -56,33 +58,34 @@ bool Heap::CreateHeapObjects() {
}
const Heap::StringTypeTable Heap::string_type_table[] = {
-#define STRING_TYPE_ELEMENT(type, size, name, camel_name) \
- {type, size, k##camel_name##MapRootIndex},
+#define STRING_TYPE_ELEMENT(type, size, name, CamelName) \
+ {type, size, RootIndex::k##CamelName##Map},
STRING_TYPE_LIST(STRING_TYPE_ELEMENT)
#undef STRING_TYPE_ELEMENT
};
const Heap::ConstantStringTable Heap::constant_string_table[] = {
- {"", kempty_stringRootIndex},
-#define CONSTANT_STRING_ELEMENT(name, contents) {contents, k##name##RootIndex},
- INTERNALIZED_STRING_LIST(CONSTANT_STRING_ELEMENT)
+ {"", RootIndex::kempty_string},
+#define CONSTANT_STRING_ELEMENT(_, name, contents) \
+ {contents, RootIndex::k##name},
+ INTERNALIZED_STRING_LIST_GENERATOR(CONSTANT_STRING_ELEMENT, /* not used */)
#undef CONSTANT_STRING_ELEMENT
};
const Heap::StructTable Heap::struct_table[] = {
-#define STRUCT_TABLE_ELEMENT(NAME, Name, name) \
- {NAME##_TYPE, Name::kSize, k##Name##MapRootIndex},
+#define STRUCT_TABLE_ELEMENT(TYPE, Name, name) \
+ {TYPE, Name::kSize, RootIndex::k##Name##Map},
STRUCT_LIST(STRUCT_TABLE_ELEMENT)
#undef STRUCT_TABLE_ELEMENT
-#define ALLOCATION_SITE_ELEMENT(NAME, Name, Size, name) \
- {NAME##_TYPE, Name::kSize##Size, k##Name##Size##MapRootIndex},
- ALLOCATION_SITE_LIST(ALLOCATION_SITE_ELEMENT)
+#define ALLOCATION_SITE_ELEMENT(_, TYPE, Name, Size, name) \
+ {TYPE, Name::kSize##Size, RootIndex::k##Name##Size##Map},
+ ALLOCATION_SITE_LIST(ALLOCATION_SITE_ELEMENT, /* not used */)
#undef ALLOCATION_SITE_ELEMENT
-#define DATA_HANDLER_ELEMENT(NAME, Name, Size, name) \
- {NAME##_TYPE, Name::kSizeWithData##Size, k##Name##Size##MapRootIndex},
- DATA_HANDLER_LIST(DATA_HANDLER_ELEMENT)
+#define DATA_HANDLER_ELEMENT(_, TYPE, Name, Size, name) \
+ {TYPE, Name::kSizeWithData##Size, RootIndex::k##Name##Size##Map},
+ DATA_HANDLER_LIST(DATA_HANDLER_ELEMENT, /* not used */)
#undef DATA_HANDLER_ELEMENT
};
@@ -91,7 +94,7 @@ AllocationResult Heap::AllocateMap(InstanceType instance_type,
ElementsKind elements_kind,
int inobject_properties) {
STATIC_ASSERT(LAST_JS_OBJECT_TYPE == LAST_TYPE);
- bool is_js_object = Map::IsJSObject(instance_type);
+ bool is_js_object = InstanceTypeChecker::IsJSObject(instance_type);
DCHECK_IMPLIES(is_js_object &&
!Map::CanHaveFastTransitionableElementsKind(instance_type),
IsDictionaryElementsKind(elements_kind) ||
@@ -119,8 +122,8 @@ AllocationResult Heap::AllocatePartialMap(InstanceType instance_type,
if (!allocation.To(&result)) return allocation;
// Map::cast cannot be used due to uninitialized map field.
Map* map = reinterpret_cast<Map*>(result);
- map->set_map_after_allocation(reinterpret_cast<Map*>(root(kMetaMapRootIndex)),
- SKIP_WRITE_BARRIER);
+ map->set_map_after_allocation(
+ reinterpret_cast<Map*>(root(RootIndex::kMetaMap)), SKIP_WRITE_BARRIER);
map->set_instance_type(instance_type);
map->set_instance_size(instance_size);
// Initialize to only containing tagged fields.
@@ -179,8 +182,9 @@ AllocationResult Heap::AllocateEmptyFixedTypedArray(
array_type == kExternalFloat64Array ? kDoubleAligned : kWordAligned);
if (!allocation.To(&object)) return allocation;
- object->set_map_after_allocation(MapForFixedTypedArray(array_type),
- SKIP_WRITE_BARRIER);
+ object->set_map_after_allocation(
+ ReadOnlyRoots(this).MapForFixedTypedArray(array_type),
+ SKIP_WRITE_BARRIER);
FixedTypedArrayBase* elements = FixedTypedArrayBase::cast(object);
elements->set_base_pointer(elements, SKIP_WRITE_BARRIER);
elements->set_external_pointer(
@@ -390,8 +394,8 @@ bool Heap::CreateInitialMaps() {
{ // Create a separate external one byte string map for native sources.
AllocationResult allocation =
- AllocateMap(SHORT_EXTERNAL_ONE_BYTE_STRING_TYPE,
- ExternalOneByteString::kShortSize);
+ AllocateMap(UNCACHED_EXTERNAL_ONE_BYTE_STRING_TYPE,
+ ExternalOneByteString::kUncachedSize);
if (!allocation.To(&obj)) return false;
Map* map = Map::cast(obj);
map->SetConstructorFunctionIndex(Context::STRING_FUNCTION_INDEX);
@@ -461,6 +465,7 @@ bool Heap::CreateInitialMaps() {
ALLOCATE_VARSIZE_MAP(CATCH_CONTEXT_TYPE, catch_context)
ALLOCATE_VARSIZE_MAP(WITH_CONTEXT_TYPE, with_context)
ALLOCATE_VARSIZE_MAP(DEBUG_EVALUATE_CONTEXT_TYPE, debug_evaluate_context)
+ ALLOCATE_VARSIZE_MAP(AWAIT_CONTEXT_TYPE, await_context)
ALLOCATE_VARSIZE_MAP(BLOCK_CONTEXT_TYPE, block_context)
ALLOCATE_VARSIZE_MAP(MODULE_CONTEXT_TYPE, module_context)
ALLOCATE_VARSIZE_MAP(EVAL_CONTEXT_TYPE, eval_context)
@@ -696,35 +701,35 @@ void Heap::CreateInitialObjects() {
{
HandleScope scope(isolate());
-#define SYMBOL_INIT(name) \
+#define SYMBOL_INIT(_, name) \
{ \
Handle<Symbol> symbol( \
isolate()->factory()->NewPrivateSymbol(TENURED_READ_ONLY)); \
- roots_[k##name##RootIndex] = *symbol; \
+ roots_[RootIndex::k##name] = *symbol; \
}
- PRIVATE_SYMBOL_LIST(SYMBOL_INIT)
+ PRIVATE_SYMBOL_LIST_GENERATOR(SYMBOL_INIT, /* not used */)
#undef SYMBOL_INIT
}
{
HandleScope scope(isolate());
-#define SYMBOL_INIT(name, description) \
+#define SYMBOL_INIT(_, name, description) \
Handle<Symbol> name = factory->NewSymbol(TENURED_READ_ONLY); \
Handle<String> name##d = \
factory->NewStringFromStaticChars(#description, TENURED_READ_ONLY); \
name->set_name(*name##d); \
- roots_[k##name##RootIndex] = *name;
- PUBLIC_SYMBOL_LIST(SYMBOL_INIT)
+ roots_[RootIndex::k##name] = *name;
+ PUBLIC_SYMBOL_LIST_GENERATOR(SYMBOL_INIT, /* not used */)
#undef SYMBOL_INIT
-#define SYMBOL_INIT(name, description) \
+#define SYMBOL_INIT(_, name, description) \
Handle<Symbol> name = factory->NewSymbol(TENURED_READ_ONLY); \
Handle<String> name##d = \
factory->NewStringFromStaticChars(#description, TENURED_READ_ONLY); \
name->set_is_well_known_symbol(true); \
name->set_name(*name##d); \
- roots_[k##name##RootIndex] = *name;
- WELL_KNOWN_SYMBOL_LIST(SYMBOL_INIT)
+ roots_[RootIndex::k##name] = *name;
+ WELL_KNOWN_SYMBOL_LIST_GENERATOR(SYMBOL_INIT, /* not used */)
#undef SYMBOL_INIT
// Mark "Interesting Symbols" appropriately.
@@ -754,9 +759,7 @@ void Heap::CreateInitialObjects() {
factory->NewManyClosuresCell(factory->undefined_value());
set_many_closures_cell(*many_closures_cell);
- // Microtask queue uses the empty fixed array as a sentinel for "empty".
- // Number of queued microtasks stored in Isolate::pending_microtask_count().
- set_microtask_queue(roots.empty_fixed_array());
+ set_default_microtask_queue(*factory->NewMicrotaskQueue());
{
Handle<FixedArray> empty_sloppy_arguments_elements =
@@ -815,6 +818,9 @@ void Heap::CreateInitialObjects() {
// Allocate the empty script.
Handle<Script> script = factory->NewScript(factory->empty_string());
script->set_type(Script::TYPE_NATIVE);
+ // This is used for exceptions thrown with no stack frames. Such exceptions
+ // can be shared everywhere.
+ script->set_origin_options(ScriptOriginOptions(true, false));
set_empty_script(*script);
Handle<Cell> array_constructor_cell = factory->NewCell(
@@ -849,6 +855,10 @@ void Heap::CreateInitialObjects() {
cell->set_value(Smi::FromInt(Isolate::kProtectorValid));
set_promise_species_protector(*cell);
+ cell = factory->NewPropertyCell(factory->empty_string());
+ cell->set_value(Smi::FromInt(Isolate::kProtectorValid));
+ set_string_iterator_protector(*cell);
+
Handle<Cell> string_length_overflow_cell = factory->NewCell(
handle(Smi::FromInt(Isolate::kProtectorValid), isolate()));
set_string_length_protector(*string_length_overflow_cell);
@@ -874,11 +884,6 @@ void Heap::CreateInitialObjects() {
set_noscript_shared_function_infos(roots.empty_weak_array_list());
- STATIC_ASSERT(interpreter::BytecodeOperands::kOperandScaleCount == 3);
- set_deserialize_lazy_handler(Smi::kZero);
- set_deserialize_lazy_handler_wide(Smi::kZero);
- set_deserialize_lazy_handler_extra_wide(Smi::kZero);
-
// Evaluate the hash values which will then be cached in the strings.
isolate()->factory()->zero_string()->Hash();
isolate()->factory()->one_string()->Hash();
@@ -901,16 +906,19 @@ void Heap::CreateInternalAccessorInfoObjects() {
HandleScope scope(isolate);
Handle<AccessorInfo> acessor_info;
-#define INIT_ACCESSOR_INFO(accessor_name, AccessorName) \
- acessor_info = Accessors::Make##AccessorName##Info(isolate); \
- roots_[k##AccessorName##AccessorRootIndex] = *acessor_info;
- ACCESSOR_INFO_LIST(INIT_ACCESSOR_INFO)
+#define INIT_ACCESSOR_INFO(_, accessor_name, AccessorName, ...) \
+ acessor_info = Accessors::Make##AccessorName##Info(isolate); \
+ roots_[RootIndex::k##AccessorName##Accessor] = *acessor_info;
+ ACCESSOR_INFO_LIST_GENERATOR(INIT_ACCESSOR_INFO, /* not used */)
#undef INIT_ACCESSOR_INFO
-#define INIT_SIDE_EFFECT_FLAG(AccessorName) \
- AccessorInfo::cast(roots_[k##AccessorName##AccessorRootIndex]) \
- ->set_has_no_side_effect(true);
- SIDE_EFFECT_FREE_ACCESSOR_INFO_LIST(INIT_SIDE_EFFECT_FLAG)
+#define INIT_SIDE_EFFECT_FLAG(_, accessor_name, AccessorName, GetterType, \
+ SetterType) \
+ AccessorInfo::cast(roots_[RootIndex::k##AccessorName##Accessor]) \
+ ->set_getter_side_effect_type(SideEffectType::GetterType); \
+ AccessorInfo::cast(roots_[RootIndex::k##AccessorName##Accessor]) \
+ ->set_setter_side_effect_type(SideEffectType::SetterType);
+ ACCESSOR_INFO_LIST_GENERATOR(INIT_SIDE_EFFECT_FLAG, /* not used */)
#undef INIT_SIDE_EFFECT_FLAG
}
diff --git a/deps/v8/src/heap/spaces-inl.h b/deps/v8/src/heap/spaces-inl.h
index 9e86905d00..7162769e5e 100644
--- a/deps/v8/src/heap/spaces-inl.h
+++ b/deps/v8/src/heap/spaces-inl.h
@@ -5,6 +5,8 @@
#ifndef V8_HEAP_SPACES_INL_H_
#define V8_HEAP_SPACES_INL_H_
+#include "src/base/atomic-utils.h"
+#include "src/base/bounded-page-allocator.h"
#include "src/base/v8-fallthrough.h"
#include "src/heap/incremental-marking.h"
#include "src/heap/spaces.h"
@@ -92,6 +94,27 @@ HeapObject* HeapObjectIterator::FromCurrentPage() {
return nullptr;
}
+void Space::IncrementExternalBackingStoreBytes(ExternalBackingStoreType type,
+ size_t amount) {
+ base::CheckedIncrement(&external_backing_store_bytes_[type], amount);
+ heap()->IncrementExternalBackingStoreBytes(type, amount);
+}
+
+void Space::DecrementExternalBackingStoreBytes(ExternalBackingStoreType type,
+ size_t amount) {
+ base::CheckedDecrement(&external_backing_store_bytes_[type], amount);
+ heap()->DecrementExternalBackingStoreBytes(type, amount);
+}
+
+void Space::MoveExternalBackingStoreBytes(ExternalBackingStoreType type,
+ Space* from, Space* to,
+ size_t amount) {
+ if (from == to) return;
+
+ base::CheckedDecrement(&(from->external_backing_store_bytes_[type]), amount);
+ base::CheckedIncrement(&(to->external_backing_store_bytes_[type]), amount);
+}
+
// -----------------------------------------------------------------------------
// SemiSpace
@@ -189,6 +212,28 @@ MemoryChunk* MemoryChunk::FromAnyPointerAddress(Heap* heap, Address addr) {
return chunk;
}
+void MemoryChunk::IncrementExternalBackingStoreBytes(
+ ExternalBackingStoreType type, size_t amount) {
+ base::CheckedIncrement(&external_backing_store_bytes_[type], amount);
+ owner()->IncrementExternalBackingStoreBytes(type, amount);
+}
+
+void MemoryChunk::DecrementExternalBackingStoreBytes(
+ ExternalBackingStoreType type, size_t amount) {
+ base::CheckedDecrement(&external_backing_store_bytes_[type], amount);
+ owner()->DecrementExternalBackingStoreBytes(type, amount);
+}
+
+void MemoryChunk::MoveExternalBackingStoreBytes(ExternalBackingStoreType type,
+ MemoryChunk* from,
+ MemoryChunk* to,
+ size_t amount) {
+ base::CheckedDecrement(&(from->external_backing_store_bytes_[type]), amount);
+ base::CheckedIncrement(&(to->external_backing_store_bytes_[type]), amount);
+ Space::MoveExternalBackingStoreBytes(type, from->owner(), to->owner(),
+ amount);
+}
+
void Page::MarkNeverAllocateForTesting() {
DCHECK(this->owner()->identity() != NEW_SPACE);
DCHECK(!IsFlagSet(NEVER_ALLOCATE_ON_PAGE));
diff --git a/deps/v8/src/heap/spaces.cc b/deps/v8/src/heap/spaces.cc
index ff28ab56b2..dcacea0afc 100644
--- a/deps/v8/src/heap/spaces.cc
+++ b/deps/v8/src/heap/spaces.cc
@@ -15,7 +15,7 @@
#include "src/heap/concurrent-marking.h"
#include "src/heap/gc-tracer.h"
#include "src/heap/heap-controller.h"
-#include "src/heap/incremental-marking.h"
+#include "src/heap/incremental-marking-inl.h"
#include "src/heap/mark-compact.h"
#include "src/heap/remembered-set.h"
#include "src/heap/slot-set.h"
@@ -94,227 +94,115 @@ PauseAllocationObserversScope::~PauseAllocationObserversScope() {
}
}
-// -----------------------------------------------------------------------------
-// CodeRange
-
static base::LazyInstance<CodeRangeAddressHint>::type code_range_address_hint =
LAZY_INSTANCE_INITIALIZER;
-CodeRange::CodeRange(Isolate* isolate, size_t requested)
+Address CodeRangeAddressHint::GetAddressHint(size_t code_range_size) {
+ base::LockGuard<base::Mutex> guard(&mutex_);
+ auto it = recently_freed_.find(code_range_size);
+ if (it == recently_freed_.end() || it->second.empty()) {
+ return reinterpret_cast<Address>(GetRandomMmapAddr());
+ }
+ Address result = it->second.back();
+ it->second.pop_back();
+ return result;
+}
+
+void CodeRangeAddressHint::NotifyFreedCodeRange(Address code_range_start,
+ size_t code_range_size) {
+ base::LockGuard<base::Mutex> guard(&mutex_);
+ recently_freed_[code_range_size].push_back(code_range_start);
+}
+
+// -----------------------------------------------------------------------------
+// MemoryAllocator
+//
+
+MemoryAllocator::MemoryAllocator(Isolate* isolate, size_t capacity,
+ size_t code_range_size)
: isolate_(isolate),
- free_list_(0),
- allocation_list_(0),
- current_allocation_block_index_(0),
- requested_code_range_size_(0) {
- DCHECK(!virtual_memory_.IsReserved());
+ data_page_allocator_(GetPlatformPageAllocator()),
+ code_page_allocator_(nullptr),
+ capacity_(RoundUp(capacity, Page::kPageSize)),
+ size_(0),
+ size_executable_(0),
+ lowest_ever_allocated_(static_cast<Address>(-1ll)),
+ highest_ever_allocated_(kNullAddress),
+ unmapper_(isolate->heap(), this) {
+ InitializeCodePageAllocator(data_page_allocator_, code_range_size);
+}
+
+void MemoryAllocator::InitializeCodePageAllocator(
+ v8::PageAllocator* page_allocator, size_t requested) {
+ DCHECK_NULL(code_page_allocator_instance_.get());
+
+ code_page_allocator_ = page_allocator;
if (requested == 0) {
+ if (!kRequiresCodeRange) return;
// When a target requires the code range feature, we put all code objects
// in a kMaximalCodeRangeSize range of virtual address space, so that
// they can call each other with near calls.
- if (kRequiresCodeRange) {
- requested = kMaximalCodeRangeSize;
- } else {
- return;
- }
- }
-
- if (requested <= kMinimumCodeRangeSize) {
+ requested = kMaximalCodeRangeSize;
+ } else if (requested <= kMinimumCodeRangeSize) {
requested = kMinimumCodeRangeSize;
}
const size_t reserved_area =
kReservedCodeRangePages * MemoryAllocator::GetCommitPageSize();
- if (requested < (kMaximalCodeRangeSize - reserved_area))
- requested += reserved_area;
-
+ if (requested < (kMaximalCodeRangeSize - reserved_area)) {
+ requested += RoundUp(reserved_area, MemoryChunk::kPageSize);
+ // Fullfilling both reserved pages requirement and huge code area
+ // alignments is not supported (requires re-implementation).
+ DCHECK_LE(kCodeRangeAreaAlignment, page_allocator->AllocatePageSize());
+ }
DCHECK(!kRequiresCodeRange || requested <= kMaximalCodeRangeSize);
- requested_code_range_size_ = requested;
-
- VirtualMemory reservation;
- void* hint = code_range_address_hint.Pointer()->GetAddressHint(requested);
- if (!AlignedAllocVirtualMemory(
- requested, Max(kCodeRangeAreaAlignment, AllocatePageSize()), hint,
- &reservation)) {
- V8::FatalProcessOutOfMemory(isolate,
+ Address hint =
+ RoundDown(code_range_address_hint.Pointer()->GetAddressHint(requested),
+ page_allocator->AllocatePageSize());
+ VirtualMemory reservation(
+ page_allocator, requested, reinterpret_cast<void*>(hint),
+ Max(kCodeRangeAreaAlignment, page_allocator->AllocatePageSize()));
+ if (!reservation.IsReserved()) {
+ V8::FatalProcessOutOfMemory(isolate_,
"CodeRange setup: allocate virtual memory");
}
+ code_range_ = reservation.region();
// We are sure that we have mapped a block of requested addresses.
DCHECK_GE(reservation.size(), requested);
Address base = reservation.address();
// On some platforms, specifically Win64, we need to reserve some pages at
- // the beginning of an executable space.
+ // the beginning of an executable space. See
+ // https://cs.chromium.org/chromium/src/components/crash/content/
+ // app/crashpad_win.cc?rcl=fd680447881449fba2edcf0589320e7253719212&l=204
+ // for details.
if (reserved_area > 0) {
if (!reservation.SetPermissions(base, reserved_area,
PageAllocator::kReadWrite))
- V8::FatalProcessOutOfMemory(isolate, "CodeRange setup: set permissions");
+ V8::FatalProcessOutOfMemory(isolate_, "CodeRange setup: set permissions");
base += reserved_area;
}
- Address aligned_base = ::RoundUp(base, MemoryChunk::kAlignment);
- size_t size = reservation.size() - (aligned_base - base) - reserved_area;
- allocation_list_.emplace_back(aligned_base, size);
- current_allocation_block_index_ = 0;
+ Address aligned_base = RoundUp(base, MemoryChunk::kAlignment);
+ size_t size =
+ RoundDown(reservation.size() - (aligned_base - base) - reserved_area,
+ MemoryChunk::kPageSize);
+ DCHECK(IsAligned(aligned_base, kCodeRangeAreaAlignment));
LOG(isolate_,
NewEvent("CodeRange", reinterpret_cast<void*>(reservation.address()),
requested));
- virtual_memory_.TakeControl(&reservation);
-}
-
-CodeRange::~CodeRange() {
- if (virtual_memory_.IsReserved()) {
- Address addr = start();
- virtual_memory_.Free();
- code_range_address_hint.Pointer()->NotifyFreedCodeRange(
- reinterpret_cast<void*>(addr), requested_code_range_size_);
- }
-}
-
-bool CodeRange::CompareFreeBlockAddress(const FreeBlock& left,
- const FreeBlock& right) {
- return left.start < right.start;
-}
-
-bool CodeRange::GetNextAllocationBlock(size_t requested) {
- for (current_allocation_block_index_++;
- current_allocation_block_index_ < allocation_list_.size();
- current_allocation_block_index_++) {
- if (requested <= allocation_list_[current_allocation_block_index_].size) {
- return true; // Found a large enough allocation block.
- }
- }
-
- // Sort and merge the free blocks on the free list and the allocation list.
- free_list_.insert(free_list_.end(), allocation_list_.begin(),
- allocation_list_.end());
- allocation_list_.clear();
- std::sort(free_list_.begin(), free_list_.end(), &CompareFreeBlockAddress);
- for (size_t i = 0; i < free_list_.size();) {
- FreeBlock merged = free_list_[i];
- i++;
- // Add adjacent free blocks to the current merged block.
- while (i < free_list_.size() &&
- free_list_[i].start == merged.start + merged.size) {
- merged.size += free_list_[i].size;
- i++;
- }
- if (merged.size > 0) {
- allocation_list_.push_back(merged);
- }
- }
- free_list_.clear();
-
- for (current_allocation_block_index_ = 0;
- current_allocation_block_index_ < allocation_list_.size();
- current_allocation_block_index_++) {
- if (requested <= allocation_list_[current_allocation_block_index_].size) {
- return true; // Found a large enough allocation block.
- }
- }
- current_allocation_block_index_ = 0;
- // Code range is full or too fragmented.
- return false;
-}
-
-
-Address CodeRange::AllocateRawMemory(const size_t requested_size,
- const size_t commit_size,
- size_t* allocated) {
- // requested_size includes the header and two guard regions, while commit_size
- // only includes the header.
- DCHECK_LE(commit_size,
- requested_size - 2 * MemoryAllocator::CodePageGuardSize());
- FreeBlock current;
- if (!ReserveBlock(requested_size, &current)) {
- *allocated = 0;
- return kNullAddress;
- }
- *allocated = current.size;
- DCHECK(IsAddressAligned(current.start, MemoryChunk::kAlignment));
- if (!isolate_->heap()->memory_allocator()->CommitExecutableMemory(
- &virtual_memory_, current.start, commit_size, *allocated)) {
- *allocated = 0;
- ReleaseBlock(&current);
- return kNullAddress;
- }
- return current.start;
-}
-
-void CodeRange::FreeRawMemory(Address address, size_t length) {
- DCHECK(IsAddressAligned(address, MemoryChunk::kAlignment));
- base::LockGuard<base::Mutex> guard(&code_range_mutex_);
- free_list_.emplace_back(address, length);
- virtual_memory_.SetPermissions(address, length, PageAllocator::kNoAccess);
-}
-
-bool CodeRange::ReserveBlock(const size_t requested_size, FreeBlock* block) {
- base::LockGuard<base::Mutex> guard(&code_range_mutex_);
- DCHECK(allocation_list_.empty() ||
- current_allocation_block_index_ < allocation_list_.size());
- if (allocation_list_.empty() ||
- requested_size > allocation_list_[current_allocation_block_index_].size) {
- // Find an allocation block large enough.
- if (!GetNextAllocationBlock(requested_size)) return false;
- }
- // Commit the requested memory at the start of the current allocation block.
- size_t aligned_requested = ::RoundUp(requested_size, MemoryChunk::kAlignment);
- *block = allocation_list_[current_allocation_block_index_];
- // Don't leave a small free block, useless for a large object or chunk.
- if (aligned_requested < (block->size - Page::kPageSize)) {
- block->size = aligned_requested;
- }
- DCHECK(IsAddressAligned(block->start, MemoryChunk::kAlignment));
- allocation_list_[current_allocation_block_index_].start += block->size;
- allocation_list_[current_allocation_block_index_].size -= block->size;
- return true;
-}
-
-
-void CodeRange::ReleaseBlock(const FreeBlock* block) {
- base::LockGuard<base::Mutex> guard(&code_range_mutex_);
- free_list_.push_back(*block);
-}
-
-void* CodeRangeAddressHint::GetAddressHint(size_t code_range_size) {
- base::LockGuard<base::Mutex> guard(&mutex_);
- auto it = recently_freed_.find(code_range_size);
- if (it == recently_freed_.end() || it->second.empty()) {
- return GetRandomMmapAddr();
- }
- void* result = it->second.back();
- it->second.pop_back();
- return result;
-}
-
-void CodeRangeAddressHint::NotifyFreedCodeRange(void* code_range_start,
- size_t code_range_size) {
- base::LockGuard<base::Mutex> guard(&mutex_);
- recently_freed_[code_range_size].push_back(code_range_start);
-}
-
-// -----------------------------------------------------------------------------
-// MemoryAllocator
-//
-
-MemoryAllocator::MemoryAllocator(Isolate* isolate, size_t capacity,
- size_t code_range_size)
- : isolate_(isolate),
- code_range_(nullptr),
- capacity_(RoundUp(capacity, Page::kPageSize)),
- size_(0),
- size_executable_(0),
- lowest_ever_allocated_(static_cast<Address>(-1ll)),
- highest_ever_allocated_(kNullAddress),
- unmapper_(isolate->heap(), this) {
- code_range_ = new CodeRange(isolate_, code_range_size);
+ heap_reservation_.TakeControl(&reservation);
+ code_page_allocator_instance_ = base::make_unique<base::BoundedPageAllocator>(
+ page_allocator, aligned_base, size,
+ static_cast<size_t>(MemoryChunk::kAlignment));
+ code_page_allocator_ = code_page_allocator_instance_.get();
}
-
void MemoryAllocator::TearDown() {
unmapper()->TearDown();
@@ -328,8 +216,15 @@ void MemoryAllocator::TearDown() {
last_chunk_.Free();
}
- delete code_range_;
- code_range_ = nullptr;
+ if (code_page_allocator_instance_.get()) {
+ DCHECK(!code_range_.is_empty());
+ code_range_address_hint.Pointer()->NotifyFreedCodeRange(code_range_.begin(),
+ code_range_.size());
+ code_range_ = base::AddressRegion();
+ code_page_allocator_instance_.reset();
+ }
+ code_page_allocator_ = nullptr;
+ data_page_allocator_ = nullptr;
}
class MemoryAllocator::Unmapper::UnmapFreeMemoryTask : public CancelableTask {
@@ -489,61 +384,41 @@ size_t MemoryAllocator::Unmapper::CommittedBufferedMemory() {
return sum;
}
-bool MemoryAllocator::CommitMemory(Address base, size_t size) {
- if (!SetPermissions(base, size, PageAllocator::kReadWrite)) {
+bool MemoryAllocator::CommitMemory(VirtualMemory* reservation) {
+ Address base = reservation->address();
+ size_t size = reservation->size();
+ if (!reservation->SetPermissions(base, size, PageAllocator::kReadWrite)) {
return false;
}
UpdateAllocatedSpaceLimits(base, base + size);
+ isolate_->counters()->memory_allocated()->Increment(static_cast<int>(size));
return true;
}
-void MemoryAllocator::FreeMemory(VirtualMemory* reservation,
- Executability executable) {
- // TODO(gc) make code_range part of memory allocator?
- // Code which is part of the code-range does not have its own VirtualMemory.
- DCHECK(code_range() == nullptr ||
- !code_range()->contains(reservation->address()));
- DCHECK(executable == NOT_EXECUTABLE || !code_range()->valid() ||
- reservation->size() <= Page::kPageSize);
-
- reservation->Free();
-}
-
-
-void MemoryAllocator::FreeMemory(Address base, size_t size,
- Executability executable) {
- // TODO(gc) make code_range part of memory allocator?
- if (code_range() != nullptr && code_range()->contains(base)) {
- DCHECK(executable == EXECUTABLE);
- code_range()->FreeRawMemory(base, size);
- } else {
- DCHECK(executable == NOT_EXECUTABLE || !code_range()->valid());
- CHECK(FreePages(reinterpret_cast<void*>(base), size));
+bool MemoryAllocator::UncommitMemory(VirtualMemory* reservation) {
+ size_t size = reservation->size();
+ if (!reservation->SetPermissions(reservation->address(), size,
+ PageAllocator::kNoAccess)) {
+ return false;
}
+ isolate_->counters()->memory_allocated()->Decrement(static_cast<int>(size));
+ return true;
}
-Address MemoryAllocator::ReserveAlignedMemory(size_t size, size_t alignment,
- void* hint,
- VirtualMemory* controller) {
- VirtualMemory reservation;
- if (!AlignedAllocVirtualMemory(size, alignment, hint, &reservation)) {
- return kNullAddress;
- }
-
- Address result = reservation.address();
- size_ += reservation.size();
- controller->TakeControl(&reservation);
- return result;
+void MemoryAllocator::FreeMemory(v8::PageAllocator* page_allocator,
+ Address base, size_t size) {
+ CHECK(FreePages(page_allocator, reinterpret_cast<void*>(base), size));
}
Address MemoryAllocator::AllocateAlignedMemory(
size_t reserve_size, size_t commit_size, size_t alignment,
Executability executable, void* hint, VirtualMemory* controller) {
+ v8::PageAllocator* page_allocator = this->page_allocator(executable);
DCHECK(commit_size <= reserve_size);
- VirtualMemory reservation;
- Address base =
- ReserveAlignedMemory(reserve_size, alignment, hint, &reservation);
- if (base == kNullAddress) return kNullAddress;
+ VirtualMemory reservation(page_allocator, reserve_size, hint, alignment);
+ if (!reservation.IsReserved()) return kNullAddress;
+ Address base = reservation.address();
+ size_ += reservation.size();
if (executable == EXECUTABLE) {
if (!CommitExecutableMemory(&reservation, base, commit_size,
@@ -608,8 +483,8 @@ void MemoryChunk::SetReadAndExecutable() {
size_t page_size = MemoryAllocator::GetCommitPageSize();
DCHECK(IsAddressAligned(protect_start, page_size));
size_t protect_size = RoundUp(area_size(), page_size);
- CHECK(SetPermissions(protect_start, protect_size,
- PageAllocator::kReadExecute));
+ CHECK(reservation_.SetPermissions(protect_start, protect_size,
+ PageAllocator::kReadExecute));
}
}
@@ -627,15 +502,15 @@ void MemoryChunk::SetReadAndWritable() {
size_t page_size = MemoryAllocator::GetCommitPageSize();
DCHECK(IsAddressAligned(unprotect_start, page_size));
size_t unprotect_size = RoundUp(area_size(), page_size);
- CHECK(SetPermissions(unprotect_start, unprotect_size,
- PageAllocator::kReadWrite));
+ CHECK(reservation_.SetPermissions(unprotect_start, unprotect_size,
+ PageAllocator::kReadWrite));
}
}
MemoryChunk* MemoryChunk::Initialize(Heap* heap, Address base, size_t size,
Address area_start, Address area_end,
Executability executable, Space* owner,
- VirtualMemory* reservation) {
+ VirtualMemory reservation) {
MemoryChunk* chunk = FromAddress(base);
DCHECK(base == chunk->address());
@@ -696,14 +571,12 @@ MemoryChunk* MemoryChunk::Initialize(Heap* heap, Address base, size_t size,
size_t page_size = MemoryAllocator::GetCommitPageSize();
DCHECK(IsAddressAligned(area_start, page_size));
size_t area_size = RoundUp(area_end - area_start, page_size);
- CHECK(SetPermissions(area_start, area_size,
- PageAllocator::kReadWriteExecute));
+ CHECK(reservation.SetPermissions(area_start, area_size,
+ PageAllocator::kReadWriteExecute));
}
}
- if (reservation != nullptr) {
- chunk->reservation_.TakeControl(reservation);
- }
+ chunk->reservation_ = std::move(reservation);
return chunk;
}
@@ -863,29 +736,12 @@ MemoryChunk* MemoryAllocator::AllocateChunk(size_t reserve_area_size,
// Size of header (not executable) plus area (executable).
size_t commit_size = ::RoundUp(
CodePageGuardStartOffset() + commit_area_size, GetCommitPageSize());
-// Allocate executable memory either from code range or from the OS.
-#ifdef V8_TARGET_ARCH_MIPS64
- // Use code range only for large object space on mips64 to keep address
- // range within 256-MB memory region.
- if (code_range()->valid() && reserve_area_size > CodePageAreaSize()) {
-#else
- if (code_range()->valid()) {
-#endif
- base =
- code_range()->AllocateRawMemory(chunk_size, commit_size, &chunk_size);
- DCHECK(IsAligned(base, MemoryChunk::kAlignment));
- if (base == kNullAddress) return nullptr;
- size_ += chunk_size;
- // Update executable memory size.
- size_executable_ += chunk_size;
- } else {
- base = AllocateAlignedMemory(chunk_size, commit_size,
- MemoryChunk::kAlignment, executable,
- address_hint, &reservation);
- if (base == kNullAddress) return nullptr;
- // Update executable memory size.
- size_executable_ += reservation.size();
- }
+ base =
+ AllocateAlignedMemory(chunk_size, commit_size, MemoryChunk::kAlignment,
+ executable, address_hint, &reservation);
+ if (base == kNullAddress) return nullptr;
+ // Update executable memory size.
+ size_executable_ += reservation.size();
if (Heap::ShouldZapGarbage()) {
ZapBlock(base, CodePageGuardStartOffset(), kZapValue);
@@ -928,7 +784,7 @@ MemoryChunk* MemoryAllocator::AllocateChunk(size_t reserve_area_size,
if ((base + chunk_size) == 0u) {
CHECK(!last_chunk_.IsReserved());
last_chunk_.TakeControl(&reservation);
- UncommitBlock(last_chunk_.address(), last_chunk_.size());
+ UncommitMemory(&last_chunk_);
size_ -= chunk_size;
if (executable == EXECUTABLE) {
size_executable_ -= chunk_size;
@@ -940,7 +796,7 @@ MemoryChunk* MemoryAllocator::AllocateChunk(size_t reserve_area_size,
MemoryChunk* chunk =
MemoryChunk::Initialize(heap, base, chunk_size, area_start, area_end,
- executable, owner, &reservation);
+ executable, owner, std::move(reservation));
if (chunk->executable()) RegisterExecutableMemoryChunk(chunk);
return chunk;
@@ -1128,12 +984,15 @@ void MemoryAllocator::PerformFreeMemory(MemoryChunk* chunk) {
VirtualMemory* reservation = chunk->reserved_memory();
if (chunk->IsFlagSet(MemoryChunk::POOLED)) {
- UncommitBlock(reinterpret_cast<Address>(chunk), MemoryChunk::kPageSize);
+ UncommitMemory(reservation);
} else {
if (reservation->IsReserved()) {
- FreeMemory(reservation, chunk->executable());
+ reservation->Free();
} else {
- FreeMemory(chunk->address(), chunk->size(), chunk->executable());
+ // Only read-only pages can have non-initialized reservation object.
+ DCHECK_EQ(RO_SPACE, chunk->owner()->identity());
+ FreeMemory(page_allocator(chunk->executable()), chunk->address(),
+ chunk->size());
}
}
}
@@ -1147,8 +1006,9 @@ void MemoryAllocator::Free(MemoryChunk* chunk) {
break;
case kAlreadyPooled:
// Pooled pages cannot be touched anymore as their memory is uncommitted.
- FreeMemory(chunk->address(), static_cast<size_t>(MemoryChunk::kPageSize),
- Executability::NOT_EXECUTABLE);
+ // Pooled pages are not-executable.
+ FreeMemory(data_page_allocator(), chunk->address(),
+ static_cast<size_t>(MemoryChunk::kPageSize));
break;
case kPooledAndQueue:
DCHECK_EQ(chunk->size(), static_cast<size_t>(MemoryChunk::kPageSize));
@@ -1216,34 +1076,19 @@ MemoryChunk* MemoryAllocator::AllocatePagePooled(SpaceType* owner) {
const Address start = reinterpret_cast<Address>(chunk);
const Address area_start = start + MemoryChunk::kObjectStartOffset;
const Address area_end = start + size;
- if (!CommitBlock(start, size)) {
- return nullptr;
+ // Pooled pages are always regular data pages.
+ DCHECK_NE(CODE_SPACE, owner->identity());
+ VirtualMemory reservation(data_page_allocator(), start, size);
+ if (!CommitMemory(&reservation)) return nullptr;
+ if (Heap::ShouldZapGarbage()) {
+ ZapBlock(start, size, kZapValue);
}
- VirtualMemory reservation(start, size);
MemoryChunk::Initialize(isolate_->heap(), start, size, area_start, area_end,
- NOT_EXECUTABLE, owner, &reservation);
+ NOT_EXECUTABLE, owner, std::move(reservation));
size_ += size;
return chunk;
}
-bool MemoryAllocator::CommitBlock(Address start, size_t size) {
- if (!CommitMemory(start, size)) return false;
-
- if (Heap::ShouldZapGarbage()) {
- ZapBlock(start, size, kZapValue);
- }
-
- isolate_->counters()->memory_allocated()->Increment(static_cast<int>(size));
- return true;
-}
-
-
-bool MemoryAllocator::UncommitBlock(Address start, size_t size) {
- if (!SetPermissions(start, size, PageAllocator::kNoAccess)) return false;
- isolate_->counters()->memory_allocated()->Decrement(static_cast<int>(size));
- return true;
-}
-
void MemoryAllocator::ZapBlock(Address start, size_t size,
uintptr_t zap_value) {
DCHECK_EQ(start % kPointerSize, 0);
@@ -1441,6 +1286,17 @@ void MemoryChunk::RegisterObjectWithInvalidatedSlots(HeapObject* object,
}
}
+bool MemoryChunk::RegisteredObjectWithInvalidatedSlots(HeapObject* object) {
+ if (ShouldSkipEvacuationSlotRecording()) {
+ // Invalidated slots do not matter if we are not recording slots.
+ return true;
+ }
+ if (invalidated_slots() == nullptr) {
+ return false;
+ }
+ return invalidated_slots()->find(object) != invalidated_slots()->end();
+}
+
void MemoryChunk::MoveObjectWithInvalidatedSlots(HeapObject* old_start,
HeapObject* new_start) {
DCHECK_LT(old_start, new_start);
@@ -1474,19 +1330,6 @@ void MemoryChunk::ReleaseYoungGenerationBitmap() {
young_generation_bitmap_ = nullptr;
}
-void MemoryChunk::IncrementExternalBackingStoreBytes(
- ExternalBackingStoreType type, size_t amount) {
- external_backing_store_bytes_[type] += amount;
- owner()->IncrementExternalBackingStoreBytes(type, amount);
-}
-
-void MemoryChunk::DecrementExternalBackingStoreBytes(
- ExternalBackingStoreType type, size_t amount) {
- DCHECK_GE(external_backing_store_bytes_[type], amount);
- external_backing_store_bytes_[type] -= amount;
- owner()->DecrementExternalBackingStoreBytes(type, amount);
-}
-
// -----------------------------------------------------------------------------
// PagedSpace implementation
@@ -2027,7 +1870,7 @@ void PagedSpace::Verify(Isolate* isolate, ObjectVisitor* visitor) {
} else if (object->IsJSArrayBuffer()) {
JSArrayBuffer* array_buffer = JSArrayBuffer::cast(object);
if (ArrayBufferTracker::IsTracked(array_buffer)) {
- size_t size = NumberToSize(array_buffer->byte_length());
+ size_t size = array_buffer->byte_length();
external_page_bytes[ExternalBackingStoreType::kArrayBuffer] += size;
}
}
@@ -2119,12 +1962,12 @@ void PagedSpace::VerifyCountersBeforeConcurrentSweeping() {
// -----------------------------------------------------------------------------
// NewSpace implementation
-NewSpace::NewSpace(Heap* heap, size_t initial_semispace_capacity,
+NewSpace::NewSpace(Heap* heap, v8::PageAllocator* page_allocator,
+ size_t initial_semispace_capacity,
size_t max_semispace_capacity)
: SpaceWithLinearArea(heap, NEW_SPACE),
to_space_(heap, kToSpace),
- from_space_(heap, kFromSpace),
- reservation_() {
+ from_space_(heap, kFromSpace) {
DCHECK(initial_semispace_capacity <= max_semispace_capacity);
DCHECK(
base::bits::IsPowerOfTwo(static_cast<uint32_t>(max_semispace_capacity)));
@@ -2515,7 +2358,7 @@ void NewSpace::Verify(Isolate* isolate) {
} else if (object->IsJSArrayBuffer()) {
JSArrayBuffer* array_buffer = JSArrayBuffer::cast(object);
if (ArrayBufferTracker::IsTracked(array_buffer)) {
- size_t size = NumberToSize(array_buffer->byte_length());
+ size_t size = array_buffer->byte_length();
external_space_bytes[ExternalBackingStoreType::kArrayBuffer] += size;
}
}
@@ -3130,7 +2973,7 @@ size_t FreeListCategory::SumFreeList() {
size_t sum = 0;
FreeSpace* cur = top();
while (cur != nullptr) {
- DCHECK(cur->map() == page()->heap()->root(Heap::kFreeSpaceMapRootIndex));
+ DCHECK(cur->map() == page()->heap()->root(RootIndex::kFreeSpaceMap));
sum += cur->relaxed_read_size();
cur = cur->next();
}
@@ -3337,12 +3180,18 @@ void ReadOnlyPage::MakeHeaderRelocatable() {
void ReadOnlySpace::SetPermissionsForPages(PageAllocator::Permission access) {
const size_t page_size = MemoryAllocator::GetCommitPageSize();
const size_t area_start_offset = RoundUp(Page::kObjectStartOffset, page_size);
+ MemoryAllocator* memory_allocator = heap()->memory_allocator();
for (Page* p : *this) {
ReadOnlyPage* page = static_cast<ReadOnlyPage*>(p);
if (access == PageAllocator::kRead) {
page->MakeHeaderRelocatable();
}
- CHECK(SetPermissions(page->address() + area_start_offset,
+
+ // Read only pages don't have valid reservation object so we get proper
+ // page allocator manually.
+ v8::PageAllocator* page_allocator =
+ memory_allocator->page_allocator(page->executable());
+ CHECK(SetPermissions(page_allocator, page->address() + area_start_offset,
page->size() - area_start_offset, access));
}
}
@@ -3473,13 +3322,7 @@ LargePage* LargeObjectSpace::AllocateLargePage(int object_size,
if (page == nullptr) return nullptr;
DCHECK_GE(page->area_size(), static_cast<size_t>(object_size));
- size_ += static_cast<int>(page->size());
- AccountCommitted(page->size());
- objects_size_ += object_size;
- page_count_++;
- memory_chunk_list_.PushBack(page);
-
- InsertChunkMapEntries(page);
+ Register(page, object_size);
HeapObject* object = page->GetObject();
@@ -3572,6 +3415,39 @@ void LargeObjectSpace::RemoveChunkMapEntries(LargePage* page,
}
}
+void LargeObjectSpace::PromoteNewLargeObject(LargePage* page) {
+ DCHECK_EQ(page->owner()->identity(), NEW_LO_SPACE);
+ DCHECK(page->IsFlagSet(MemoryChunk::IN_FROM_SPACE));
+ DCHECK(!page->IsFlagSet(MemoryChunk::IN_TO_SPACE));
+ size_t object_size = static_cast<size_t>(page->GetObject()->Size());
+ reinterpret_cast<NewLargeObjectSpace*>(page->owner())
+ ->Unregister(page, object_size);
+ Register(page, object_size);
+ page->ClearFlag(MemoryChunk::IN_FROM_SPACE);
+ page->SetOldGenerationPageFlags(heap()->incremental_marking()->IsMarking());
+ page->set_owner(this);
+}
+
+void LargeObjectSpace::Register(LargePage* page, size_t object_size) {
+ size_ += static_cast<int>(page->size());
+ AccountCommitted(page->size());
+ objects_size_ += object_size;
+ page_count_++;
+ memory_chunk_list_.PushBack(page);
+
+ InsertChunkMapEntries(page);
+}
+
+void LargeObjectSpace::Unregister(LargePage* page, size_t object_size) {
+ size_ -= static_cast<int>(page->size());
+ AccountUncommitted(page->size());
+ objects_size_ -= object_size;
+ page_count_--;
+ memory_chunk_list_.Remove(page);
+
+ RemoveChunkMapEntries(page);
+}
+
void LargeObjectSpace::FreeUnmarkedObjects() {
LargePage* current = first_page();
IncrementalMarking::NonAtomicMarkingState* marking_state =
@@ -3759,5 +3635,13 @@ size_t NewLargeObjectSpace::Available() {
// TODO(hpayer): Update as soon as we have a growing strategy.
return 0;
}
+
+void NewLargeObjectSpace::Flip() {
+ for (LargePage* chunk = first_page(); chunk != nullptr;
+ chunk = chunk->next_page()) {
+ chunk->SetFlag(MemoryChunk::IN_FROM_SPACE);
+ chunk->ClearFlag(MemoryChunk::IN_TO_SPACE);
+ }
+}
} // namespace internal
} // namespace v8
diff --git a/deps/v8/src/heap/spaces.h b/deps/v8/src/heap/spaces.h
index 47272501f3..018e9da47b 100644
--- a/deps/v8/src/heap/spaces.h
+++ b/deps/v8/src/heap/spaces.h
@@ -14,6 +14,7 @@
#include "src/allocation.h"
#include "src/base/atomic-utils.h"
+#include "src/base/bounded-page-allocator.h"
#include "src/base/iterator.h"
#include "src/base/list.h"
#include "src/base/platform/mutex.h"
@@ -32,7 +33,7 @@ namespace internal {
namespace heap {
class HeapTester;
-class TestCodeRangeScope;
+class TestCodePageAllocatorScope;
} // namespace heap
class AllocationObserver;
@@ -109,11 +110,10 @@ class Space;
// Some assertion macros used in the debugging mode.
-#define DCHECK_PAGE_ALIGNED(address) \
- DCHECK((OffsetFrom(address) & kPageAlignmentMask) == 0)
+#define DCHECK_PAGE_ALIGNED(address) DCHECK_EQ(0, (address)&kPageAlignmentMask)
#define DCHECK_OBJECT_ALIGNED(address) \
- DCHECK((OffsetFrom(address) & kObjectAlignmentMask) == 0)
+ DCHECK_EQ(0, (address)&kObjectAlignmentMask)
#define DCHECK_OBJECT_SIZE(size) \
DCHECK((0 < size) && (size <= kMaxRegularHeapObjectSize))
@@ -142,12 +142,6 @@ enum FreeMode { kLinkCategory, kDoNotLinkCategory };
enum class SpaceAccountingMode { kSpaceAccounted, kSpaceUnaccounted };
-enum ExternalBackingStoreType {
- kArrayBuffer,
- kExternalString,
- kNumTypes
-};
-
enum RememberedSetType {
OLD_TO_NEW,
OLD_TO_OLD,
@@ -363,7 +357,7 @@ class MemoryChunk {
+ kUIntptrSize // uintptr_t flags_
+ kPointerSize // Address area_start_
+ kPointerSize // Address area_end_
- + 2 * kPointerSize // VirtualMemory reservation_
+ + 3 * kPointerSize // VirtualMemory reservation_
+ kPointerSize // Address owner_
+ kPointerSize // Heap* heap_
+ kIntptrSize // intptr_t progress_bar_
@@ -378,7 +372,7 @@ class MemoryChunk {
kPointerSize // std::atomic<ConcurrentSweepingState> concurrent_sweeping_
+ kPointerSize // base::Mutex* page_protection_change_mutex_
+ kPointerSize // unitptr_t write_unprotect_counter_
- + kSizetSize * kNumTypes
+ + kSizetSize * ExternalBackingStoreType::kNumTypes
// std::atomic<size_t> external_backing_store_bytes_
+ kSizetSize // size_t allocated_bytes_
+ kSizetSize // size_t wasted_memory_
@@ -416,7 +410,7 @@ class MemoryChunk {
// Only works if the pointer is in the first kPageSize of the MemoryChunk.
static MemoryChunk* FromAddress(Address a) {
- return reinterpret_cast<MemoryChunk*>(OffsetFrom(a) & ~kAlignmentMask);
+ return reinterpret_cast<MemoryChunk*>(a & ~kAlignmentMask);
}
// Only works if the object is in the first kPageSize of the MemoryChunk.
static MemoryChunk* FromHeapObject(const HeapObject* o) {
@@ -444,6 +438,10 @@ class MemoryChunk {
!chunk->high_water_mark_.compare_exchange_weak(old_mark, new_mark));
}
+ static inline void MoveExternalBackingStoreBytes(
+ ExternalBackingStoreType type, MemoryChunk* from, MemoryChunk* to,
+ size_t amount);
+
Address address() const {
return reinterpret_cast<Address>(const_cast<MemoryChunk*>(this));
}
@@ -518,6 +516,7 @@ class MemoryChunk {
// Updates invalidated_slots after array left-trimming.
void MoveObjectWithInvalidatedSlots(HeapObject* old_start,
HeapObject* new_start);
+ bool RegisteredObjectWithInvalidatedSlots(HeapObject* object);
InvalidatedSlots* invalidated_slots() { return invalidated_slots_; }
void ReleaseLocalTracker();
@@ -550,10 +549,12 @@ class MemoryChunk {
}
}
- void IncrementExternalBackingStoreBytes(ExternalBackingStoreType type,
- size_t amount);
- void DecrementExternalBackingStoreBytes(ExternalBackingStoreType type,
- size_t amount);
+ inline void IncrementExternalBackingStoreBytes(ExternalBackingStoreType type,
+ size_t amount);
+
+ inline void DecrementExternalBackingStoreBytes(ExternalBackingStoreType type,
+ size_t amount);
+
size_t ExternalBackingStoreBytes(ExternalBackingStoreType type) {
return external_backing_store_bytes_[type];
}
@@ -652,7 +653,7 @@ class MemoryChunk {
static MemoryChunk* Initialize(Heap* heap, Address base, size_t size,
Address area_start, Address area_end,
Executability executable, Space* owner,
- VirtualMemory* reservation);
+ VirtualMemory reservation);
// Should be called when memory chunk is about to be freed.
void ReleaseAllocatedMemory();
@@ -775,7 +776,7 @@ class Page : public MemoryChunk {
// from [page_addr .. page_addr + kPageSize[. This only works if the object
// is in fact in a page.
static Page* FromAddress(Address addr) {
- return reinterpret_cast<Page*>(OffsetFrom(addr) & ~kPageAlignmentMask);
+ return reinterpret_cast<Page*>(addr & ~kPageAlignmentMask);
}
static Page* FromHeapObject(const HeapObject* o) {
return reinterpret_cast<Page*>(reinterpret_cast<Address>(o) &
@@ -797,7 +798,7 @@ class Page : public MemoryChunk {
// Checks whether an address is page aligned.
static bool IsAlignedToPageSize(Address addr) {
- return (OffsetFrom(addr) & kPageAlignmentMask) == 0;
+ return (addr & kPageAlignmentMask) == 0;
}
static bool IsAtObjectStart(Address addr) {
@@ -894,10 +895,23 @@ class ReadOnlyPage : public Page {
// Clears any pointers in the header that point out of the page that would
// otherwise make the header non-relocatable.
void MakeHeaderRelocatable();
+
+ private:
+ friend class ReadOnlySpace;
};
class LargePage : public MemoryChunk {
public:
+ // A limit to guarantee that we do not overflow typed slot offset in
+ // the old to old remembered set.
+ // Note that this limit is higher than what assembler already imposes on
+ // x64 and ia32 architectures.
+ static const int kMaxCodePageSize = 512 * MB;
+
+ static LargePage* FromHeapObject(const HeapObject* o) {
+ return static_cast<LargePage*>(MemoryChunk::FromHeapObject(o));
+ }
+
HeapObject* GetObject() { return HeapObject::FromAddress(area_start()); }
inline LargePage* next_page() {
@@ -910,12 +924,6 @@ class LargePage : public MemoryChunk {
void ClearOutOfLiveRangeSlots(Address free_start);
- // A limit to guarantee that we do not overflow typed slot offset in
- // the old to old remembered set.
- // Note that this limit is higher than what assembler already imposes on
- // x64 and ia32 architectures.
- static const int kMaxCodePageSize = 512 * MB;
-
private:
static LargePage* Initialize(Heap* heap, MemoryChunk* chunk,
Executability executable);
@@ -941,6 +949,9 @@ class Space : public Malloced {
0;
}
+ static inline void MoveExternalBackingStoreBytes(
+ ExternalBackingStoreType type, Space* from, Space* to, size_t amount);
+
virtual ~Space() {
delete[] external_backing_store_bytes_;
external_backing_store_bytes_ = nullptr;
@@ -980,12 +991,6 @@ class Space : public Malloced {
// (e.g. see LargeObjectSpace).
virtual size_t SizeOfObjects() { return Size(); }
- // Returns amount of off-heap memory in-use by objects in this Space.
- virtual size_t ExternalBackingStoreBytes(
- ExternalBackingStoreType type) const {
- return external_backing_store_bytes_[type];
- }
-
// Approximate amount of physical memory committed for this space.
virtual size_t CommittedPhysicalMemory() = 0;
@@ -1015,14 +1020,16 @@ class Space : public Malloced {
committed_ -= bytes;
}
- void IncrementExternalBackingStoreBytes(ExternalBackingStoreType type,
- size_t amount) {
- external_backing_store_bytes_[type] += amount;
- }
- void DecrementExternalBackingStoreBytes(ExternalBackingStoreType type,
- size_t amount) {
- DCHECK_GE(external_backing_store_bytes_[type], amount);
- external_backing_store_bytes_[type] -= amount;
+ inline void IncrementExternalBackingStoreBytes(ExternalBackingStoreType type,
+ size_t amount);
+
+ inline void DecrementExternalBackingStoreBytes(ExternalBackingStoreType type,
+ size_t amount);
+
+ // Returns amount of off-heap memory in-use by objects in this Space.
+ virtual size_t ExternalBackingStoreBytes(
+ ExternalBackingStoreType type) const {
+ return external_backing_store_bytes_[type];
}
V8_EXPORT_PRIVATE void* GetRandomMmapAddr();
@@ -1074,94 +1081,6 @@ class MemoryChunkValidator {
};
-// ----------------------------------------------------------------------------
-// All heap objects containing executable code (code objects) must be allocated
-// from a 2 GB range of memory, so that they can call each other using 32-bit
-// displacements. This happens automatically on 32-bit platforms, where 32-bit
-// displacements cover the entire 4GB virtual address space. On 64-bit
-// platforms, we support this using the CodeRange object, which reserves and
-// manages a range of virtual memory.
-class CodeRange {
- public:
- CodeRange(Isolate* isolate, size_t requested_size);
- ~CodeRange();
-
- bool valid() { return virtual_memory_.IsReserved(); }
- Address start() {
- DCHECK(valid());
- return virtual_memory_.address();
- }
- size_t size() {
- DCHECK(valid());
- return virtual_memory_.size();
- }
- bool contains(Address address) {
- if (!valid()) return false;
- Address start = virtual_memory_.address();
- return start <= address && address < start + virtual_memory_.size();
- }
-
- // Allocates a chunk of memory from the large-object portion of
- // the code range. On platforms with no separate code range, should
- // not be called.
- V8_WARN_UNUSED_RESULT Address AllocateRawMemory(const size_t requested_size,
- const size_t commit_size,
- size_t* allocated);
- void FreeRawMemory(Address buf, size_t length);
-
- private:
- class FreeBlock {
- public:
- FreeBlock() : start(0), size(0) {}
- FreeBlock(Address start_arg, size_t size_arg)
- : start(start_arg), size(size_arg) {
- DCHECK(IsAddressAligned(start, MemoryChunk::kAlignment));
- DCHECK(size >= static_cast<size_t>(Page::kPageSize));
- }
- FreeBlock(void* start_arg, size_t size_arg)
- : start(reinterpret_cast<Address>(start_arg)), size(size_arg) {
- DCHECK(IsAddressAligned(start, MemoryChunk::kAlignment));
- DCHECK(size >= static_cast<size_t>(Page::kPageSize));
- }
-
- Address start;
- size_t size;
- };
-
- // Finds a block on the allocation list that contains at least the
- // requested amount of memory. If none is found, sorts and merges
- // the existing free memory blocks, and searches again.
- // If none can be found, returns false.
- bool GetNextAllocationBlock(size_t requested);
- // Compares the start addresses of two free blocks.
- static bool CompareFreeBlockAddress(const FreeBlock& left,
- const FreeBlock& right);
- bool ReserveBlock(const size_t requested_size, FreeBlock* block);
- void ReleaseBlock(const FreeBlock* block);
-
- Isolate* isolate_;
-
- // The reserved range of virtual memory that all code objects are put in.
- VirtualMemory virtual_memory_;
-
- // The global mutex guards free_list_ and allocation_list_ as GC threads may
- // access both lists concurrently to the main thread.
- base::Mutex code_range_mutex_;
-
- // Freed blocks of memory are added to the free list. When the allocation
- // list is exhausted, the free list is sorted and merged to make the new
- // allocation list.
- std::vector<FreeBlock> free_list_;
-
- // Memory is allocated from the free blocks on the allocation list.
- // The block at current_allocation_block_index_ is the current block.
- std::vector<FreeBlock> allocation_list_;
- size_t current_allocation_block_index_;
- size_t requested_code_range_size_;
-
- DISALLOW_COPY_AND_ASSIGN(CodeRange);
-};
-
// The process-wide singleton that keeps track of code range regions with the
// intention to reuse free code range regions as a workaround for CFG memory
// leaks (see crbug.com/870054).
@@ -1169,9 +1088,9 @@ class CodeRangeAddressHint {
public:
// Returns the most recently freed code range start address for the given
// size. If there is no such entry, then a random address is returned.
- V8_EXPORT_PRIVATE void* GetAddressHint(size_t code_range_size);
+ V8_EXPORT_PRIVATE Address GetAddressHint(size_t code_range_size);
- V8_EXPORT_PRIVATE void NotifyFreedCodeRange(void* code_range_start,
+ V8_EXPORT_PRIVATE void NotifyFreedCodeRange(Address code_range_start,
size_t code_range_size);
private:
@@ -1180,7 +1099,7 @@ class CodeRangeAddressHint {
// addresses. There should be O(1) different code range sizes.
// The length of each array is limited by the peak number of code ranges,
// which should be also O(1).
- std::map<size_t, std::vector<void*>> recently_freed_;
+ std::unordered_map<size_t, std::vector<Address>> recently_freed_;
};
class SkipList {
@@ -1211,7 +1130,7 @@ class SkipList {
}
static inline int RegionNumber(Address addr) {
- return (OffsetFrom(addr) & kPageAlignmentMask) >> kRegionSizeLog2;
+ return (addr & kPageAlignmentMask) >> kRegionSizeLog2;
}
static void Update(Address addr, int size) {
@@ -1422,16 +1341,11 @@ class V8_EXPORT_PRIVATE MemoryAllocator {
MemoryChunk* AllocateChunk(size_t reserve_area_size, size_t commit_area_size,
Executability executable, Space* space);
- Address ReserveAlignedMemory(size_t requested, size_t alignment, void* hint,
- VirtualMemory* controller);
Address AllocateAlignedMemory(size_t reserve_size, size_t commit_size,
size_t alignment, Executability executable,
void* hint, VirtualMemory* controller);
- bool CommitMemory(Address addr, size_t size);
-
- void FreeMemory(VirtualMemory* reservation, Executability executable);
- void FreeMemory(Address addr, size_t size, Executability executable);
+ void FreeMemory(v8::PageAllocator* page_allocator, Address addr, size_t size);
// Partially release |bytes_to_free| bytes starting at |start_free|. Note that
// internally memory is freed from |start_free| to the end of the reservation.
@@ -1440,23 +1354,19 @@ class V8_EXPORT_PRIVATE MemoryAllocator {
void PartialFreeMemory(MemoryChunk* chunk, Address start_free,
size_t bytes_to_free, Address new_area_end);
- // Commit a contiguous block of memory from the initial chunk. Assumes that
- // the address is not kNullAddress, the size is greater than zero, and that
- // the block is contained in the initial chunk. Returns true if it succeeded
- // and false otherwise.
- bool CommitBlock(Address start, size_t size);
-
// Checks if an allocated MemoryChunk was intended to be used for executable
// memory.
bool IsMemoryChunkExecutable(MemoryChunk* chunk) {
return executable_memory_.find(chunk) != executable_memory_.end();
}
- // Uncommit a contiguous block of memory [start..(start+size)[.
- // start is not kNullAddress, the size is greater than zero, and the
- // block is contained in the initial chunk. Returns true if it succeeded
- // and false otherwise.
- bool UncommitBlock(Address start, size_t size);
+ // Commit memory region owned by given reservation object. Returns true if
+ // it succeeded and false otherwise.
+ bool CommitMemory(VirtualMemory* reservation);
+
+ // Uncommit memory region owned by given reservation object. Returns true if
+ // it succeeded and false otherwise.
+ bool UncommitMemory(VirtualMemory* reservation);
// Zaps a contiguous block of memory [start..(start+size)[ with
// a given zap value.
@@ -1467,10 +1377,38 @@ class V8_EXPORT_PRIVATE MemoryAllocator {
size_t commit_size,
size_t reserved_size);
- CodeRange* code_range() { return code_range_; }
+ // Page allocator instance for allocating non-executable pages.
+ // Guaranteed to be a valid pointer.
+ v8::PageAllocator* data_page_allocator() { return data_page_allocator_; }
+
+ // Page allocator instance for allocating executable pages.
+ // Guaranteed to be a valid pointer.
+ v8::PageAllocator* code_page_allocator() { return code_page_allocator_; }
+
+ // Returns page allocator suitable for allocating pages with requested
+ // executability.
+ v8::PageAllocator* page_allocator(Executability executable) {
+ return executable == EXECUTABLE ? code_page_allocator_
+ : data_page_allocator_;
+ }
+
+ // A region of memory that may contain executable code including reserved
+ // OS page with read-write access in the beginning.
+ const base::AddressRegion& code_range() const {
+ // |code_range_| >= |optional RW pages| + |code_page_allocator_instance_|
+ DCHECK_IMPLIES(!code_range_.is_empty(), code_page_allocator_instance_);
+ DCHECK_IMPLIES(!code_range_.is_empty(),
+ code_range_.contains(code_page_allocator_instance_->begin(),
+ code_page_allocator_instance_->size()));
+ return code_range_;
+ }
+
Unmapper* unmapper() { return &unmapper_; }
private:
+ void InitializeCodePageAllocator(v8::PageAllocator* page_allocator,
+ size_t requested);
+
// PreFree logically frees the object, i.e., it takes care of the size
// bookkeeping and calls the allocation callback.
void PreFreeMemory(MemoryChunk* chunk);
@@ -1518,7 +1456,43 @@ class V8_EXPORT_PRIVATE MemoryAllocator {
}
Isolate* isolate_;
- CodeRange* code_range_;
+
+ // This object controls virtual space reserved for V8 heap instance.
+ // Depending on the configuration it may contain the following:
+ // - no reservation (on 32-bit architectures)
+ // - code range reservation used by bounded code page allocator (on 64-bit
+ // architectures without pointers compression in V8 heap)
+ // - data + code range reservation (on 64-bit architectures with pointers
+ // compression in V8 heap)
+ VirtualMemory heap_reservation_;
+
+ // Page allocator used for allocating data pages. Depending on the
+ // configuration it may be a page allocator instance provided by v8::Platform
+ // or a BoundedPageAllocator (when pointer compression is enabled).
+ v8::PageAllocator* data_page_allocator_;
+
+ // Page allocator used for allocating code pages. Depending on the
+ // configuration it may be a page allocator instance provided by v8::Platform
+ // or a BoundedPageAllocator (when pointer compression is enabled or
+ // on those 64-bit architectures where pc-relative 32-bit displacement
+ // can be used for call and jump instructions).
+ v8::PageAllocator* code_page_allocator_;
+
+ // A part of the |heap_reservation_| that may contain executable code
+ // including reserved page with read-write access in the beginning.
+ // See details below.
+ base::AddressRegion code_range_;
+
+ // This unique pointer owns the instance of bounded code allocator
+ // that controls executable pages allocation. It does not control the
+ // optionally existing page in the beginning of the |code_range_|.
+ // So, summarizing all above, the following conditions hold:
+ // 1) |heap_reservation_| >= |code_range_|
+ // 2) |code_range_| >= |optional RW pages| + |code_page_allocator_instance_|.
+ // 3) |heap_reservation_| is AllocatePageSize()-aligned
+ // 4) |code_page_allocator_instance_| is MemoryChunk::kAlignment-aligned
+ // 5) |code_range_| is CommitPageSize()-aligned
+ std::unique_ptr<base::BoundedPageAllocator> code_page_allocator_instance_;
// Maximum space size in bytes.
size_t capacity_;
@@ -1542,7 +1516,7 @@ class V8_EXPORT_PRIVATE MemoryAllocator {
// Data structure to remember allocated executable memory chunks.
std::unordered_set<MemoryChunk*> executable_memory_;
- friend class heap::TestCodeRangeScope;
+ friend class heap::TestCodePageAllocatorScope;
DISALLOW_IMPLICIT_CONSTRUCTORS(MemoryAllocator);
};
@@ -1567,7 +1541,7 @@ MemoryAllocator::AllocatePage<MemoryAllocator::kPooled, SemiSpace>(
class V8_EXPORT_PRIVATE ObjectIterator : public Malloced {
public:
- virtual ~ObjectIterator() {}
+ virtual ~ObjectIterator() = default;
virtual HeapObject* Next() = 0;
};
@@ -1701,7 +1675,7 @@ class LinearAllocationArea {
// functions increase or decrease one of the non-capacity stats in conjunction
// with capacity, or else they always balance increases and decreases to the
// non-capacity stats.
-class AllocationStats BASE_EMBEDDED {
+class AllocationStats {
public:
AllocationStats() { Clear(); }
@@ -2604,8 +2578,8 @@ class NewSpace : public SpaceWithLinearArea {
public:
typedef PageIterator iterator;
- NewSpace(Heap* heap, size_t initial_semispace_capacity,
- size_t max_semispace_capacity);
+ NewSpace(Heap* heap, v8::PageAllocator* page_allocator,
+ size_t initial_semispace_capacity, size_t max_semispace_capacity);
~NewSpace() override { TearDown(); }
@@ -3037,6 +3011,8 @@ class LargeObjectSpace : public Space {
void RemoveChunkMapEntries(LargePage* page);
void RemoveChunkMapEntries(LargePage* page, Address free_start);
+ void PromoteNewLargeObject(LargePage* page);
+
// Checks whether a heap object is in this space; O(1).
bool Contains(HeapObject* obj);
// Checks whether an address is in the object area in this space. Iterates
@@ -3046,6 +3022,9 @@ class LargeObjectSpace : public Space {
// Checks whether the space is empty.
bool IsEmpty() { return first_page() == nullptr; }
+ void Register(LargePage* page, size_t object_size);
+ void Unregister(LargePage* page, size_t object_size);
+
LargePage* first_page() {
return reinterpret_cast<LargePage*>(Space::first_page());
}
@@ -3094,6 +3073,8 @@ class NewLargeObjectSpace : public LargeObjectSpace {
// Available bytes for objects in this space.
size_t Available() override;
+
+ void Flip();
};
class LargeObjectIterator : public ObjectIterator {
@@ -3108,7 +3089,7 @@ class LargeObjectIterator : public ObjectIterator {
// Iterates over the chunks (pages and large object pages) that can contain
// pointers to new space or to evacuation candidates.
-class MemoryChunkIterator BASE_EMBEDDED {
+class MemoryChunkIterator {
public:
inline explicit MemoryChunkIterator(Heap* heap);
diff --git a/deps/v8/src/heap/store-buffer.cc b/deps/v8/src/heap/store-buffer.cc
index b428a82046..f737eb099d 100644
--- a/deps/v8/src/heap/store-buffer.cc
+++ b/deps/v8/src/heap/store-buffer.cc
@@ -31,15 +31,15 @@ StoreBuffer::StoreBuffer(Heap* heap)
}
void StoreBuffer::SetUp() {
+ v8::PageAllocator* page_allocator = GetPlatformPageAllocator();
const size_t requested_size = kStoreBufferSize * kStoreBuffers;
// Allocate buffer memory aligned at least to kStoreBufferSize. This lets us
// use a bit test to detect the ends of the buffers.
const size_t alignment =
- std::max<size_t>(kStoreBufferSize, AllocatePageSize());
+ std::max<size_t>(kStoreBufferSize, page_allocator->AllocatePageSize());
void* hint = AlignedAddress(heap_->GetRandomMmapAddr(), alignment);
- VirtualMemory reservation;
- if (!AlignedAllocVirtualMemory(requested_size, alignment, hint,
- &reservation)) {
+ VirtualMemory reservation(page_allocator, requested_size, hint, alignment);
+ if (!reservation.IsReserved()) {
heap_->FatalProcessOutOfMemory("StoreBuffer::SetUp");
}
diff --git a/deps/v8/src/heap/store-buffer.h b/deps/v8/src/heap/store-buffer.h
index d2c0f9b75f..4dbb471b7a 100644
--- a/deps/v8/src/heap/store-buffer.h
+++ b/deps/v8/src/heap/store-buffer.h
@@ -123,7 +123,7 @@ class StoreBuffer {
: CancelableTask(isolate),
store_buffer_(store_buffer),
tracer_(isolate->heap()->tracer()) {}
- virtual ~Task() {}
+ ~Task() override = default;
private:
void RunInternal() override {
diff --git a/deps/v8/src/heap/sweeper.cc b/deps/v8/src/heap/sweeper.cc
index 9e622c3385..4f5ad18bec 100644
--- a/deps/v8/src/heap/sweeper.cc
+++ b/deps/v8/src/heap/sweeper.cc
@@ -76,7 +76,7 @@ class Sweeper::SweeperTask final : public CancelableTask {
space_to_start_(space_to_start),
tracer_(isolate->heap()->tracer()) {}
- virtual ~SweeperTask() {}
+ ~SweeperTask() override = default;
private:
void RunInternal() final {
@@ -111,7 +111,7 @@ class Sweeper::IncrementalSweeperTask final : public CancelableTask {
IncrementalSweeperTask(Isolate* isolate, Sweeper* sweeper)
: CancelableTask(isolate), isolate_(isolate), sweeper_(sweeper) {}
- virtual ~IncrementalSweeperTask() {}
+ ~IncrementalSweeperTask() override = default;
private:
void RunInternal() final {
@@ -447,10 +447,11 @@ int Sweeper::ParallelSweepPage(Page* page, AllocationSpace identity) {
void Sweeper::ScheduleIncrementalSweepingTask() {
if (!incremental_sweeper_pending_) {
incremental_sweeper_pending_ = true;
- IncrementalSweeperTask* task =
- new IncrementalSweeperTask(heap_->isolate(), this);
v8::Isolate* isolate = reinterpret_cast<v8::Isolate*>(heap_->isolate());
- V8::GetCurrentPlatform()->CallOnForegroundThread(isolate, task);
+ auto taskrunner =
+ V8::GetCurrentPlatform()->GetForegroundTaskRunner(isolate);
+ taskrunner->PostTask(
+ base::make_unique<IncrementalSweeperTask>(heap_->isolate(), this));
}
}
@@ -530,7 +531,7 @@ class Sweeper::IterabilityTask final : public CancelableTask {
pending_iterability_task_(pending_iterability_task),
tracer_(isolate->heap()->tracer()) {}
- virtual ~IterabilityTask() {}
+ ~IterabilityTask() override = default;
private:
void RunInternal() final {