< prev index next >

src/hotspot/share/memory/metaspace/virtualSpaceNode.cpp

Print this page
rev 57625 : [mq]: metaspace-improvement

*** 20,588 **** * or visit www.oracle.com if you need additional information or have any * questions. * */ #include "precompiled.hpp" #include "logging/log.hpp" ! #include "logging/logStream.hpp" #include "memory/metaspace/metachunk.hpp" - #include "memory/metaspace.hpp" - #include "memory/metaspace/chunkManager.hpp" - #include "memory/metaspace/metaDebug.hpp" #include "memory/metaspace/metaspaceCommon.hpp" ! #include "memory/metaspace/occupancyMap.hpp" #include "memory/metaspace/virtualSpaceNode.hpp" ! #include "memory/virtualspace.hpp" #include "runtime/os.hpp" ! #include "services/memTracker.hpp" ! #include "utilities/copy.hpp" #include "utilities/debug.hpp" #include "utilities/globalDefinitions.hpp" namespace metaspace { ! // Decide if large pages should be committed when the memory is reserved. ! static bool should_commit_large_pages_when_reserving(size_t bytes) { ! if (UseLargePages && UseLargePagesInMetaspace && !os::can_commit_large_page_memory()) { ! size_t words = bytes / BytesPerWord; ! bool is_class = false; // We never reserve large pages for the class space. ! if (MetaspaceGC::can_expand(words, is_class) && ! MetaspaceGC::allowed_expansion() >= words) { ! return true; ! } ! } ! ! return false; } ! // byte_size is the size of the associated virtualspace. ! VirtualSpaceNode::VirtualSpaceNode(bool is_class, size_t bytes) : ! _next(NULL), _is_class(is_class), _rs(), _top(NULL), _container_count(0), _occupancy_map(NULL) { ! assert_is_aligned(bytes, Metaspace::reserve_alignment()); ! bool large_pages = should_commit_large_pages_when_reserving(bytes); ! _rs = ReservedSpace(bytes, Metaspace::reserve_alignment(), large_pages); ! ! if (_rs.is_reserved()) { ! assert(_rs.base() != NULL, "Catch if we get a NULL address"); ! assert(_rs.size() != 0, "Catch if we get a 0 size"); ! assert_is_aligned(_rs.base(), Metaspace::reserve_alignment()); ! assert_is_aligned(_rs.size(), Metaspace::reserve_alignment()); ! MemTracker::record_virtual_memory_type((address)_rs.base(), mtClass); ! } ! } ! void VirtualSpaceNode::purge(ChunkManager* chunk_manager) { ! // When a node is purged, lets give it a thorough examination. ! DEBUG_ONLY(verify(true);) ! Metachunk* chunk = first_chunk(); ! Metachunk* invalid_chunk = (Metachunk*) top(); ! while (chunk < invalid_chunk ) { ! assert(chunk->is_tagged_free(), "Should be tagged free"); ! MetaWord* next = ((MetaWord*)chunk) + chunk->word_size(); ! chunk_manager->remove_chunk(chunk); ! chunk->remove_sentinel(); ! assert(chunk->next() == NULL && ! chunk->prev() == NULL, ! "Was not removed from its list"); ! chunk = (Metachunk*) next; ! } ! } ! void VirtualSpaceNode::print_map(outputStream* st, bool is_class) const { ! if (bottom() == top()) { ! return; } ! const size_t spec_chunk_size = is_class ? ClassSpecializedChunk : SpecializedChunk; ! const size_t small_chunk_size = is_class ? ClassSmallChunk : SmallChunk; ! const size_t med_chunk_size = is_class ? ClassMediumChunk : MediumChunk; ! ! int line_len = 100; ! const size_t section_len = align_up(spec_chunk_size * line_len, med_chunk_size); ! line_len = (int)(section_len / spec_chunk_size); ! ! static const int NUM_LINES = 4; ! ! char* lines[NUM_LINES]; ! for (int i = 0; i < NUM_LINES; i ++) { ! lines[i] = (char*)os::malloc(line_len, mtInternal); ! } ! int pos = 0; ! const MetaWord* p = bottom(); ! const Metachunk* chunk = (const Metachunk*)p; ! const MetaWord* chunk_end = p + chunk->word_size(); ! while (p < top()) { ! if (pos == line_len) { ! pos = 0; ! for (int i = 0; i < NUM_LINES; i ++) { ! st->fill_to(22); ! st->print_raw(lines[i], line_len); ! st->cr(); ! } ! } ! if (pos == 0) { ! st->print(PTR_FORMAT ":", p2i(p)); ! } ! if (p == chunk_end) { ! chunk = (Metachunk*)p; ! chunk_end = p + chunk->word_size(); } - // line 1: chunk starting points (a dot if that area is a chunk start). - lines[0][pos] = p == (const MetaWord*)chunk ? '.' : ' '; ! // Line 2: chunk type (x=spec, s=small, m=medium, h=humongous), uppercase if ! // chunk is in use. ! const bool chunk_is_free = ((Metachunk*)chunk)->is_tagged_free(); ! if (chunk->word_size() == spec_chunk_size) { ! lines[1][pos] = chunk_is_free ? 'x' : 'X'; ! } else if (chunk->word_size() == small_chunk_size) { ! lines[1][pos] = chunk_is_free ? 's' : 'S'; ! } else if (chunk->word_size() == med_chunk_size) { ! lines[1][pos] = chunk_is_free ? 'm' : 'M'; ! } else if (chunk->word_size() > med_chunk_size) { ! lines[1][pos] = chunk_is_free ? 'h' : 'H'; ! } else { ! ShouldNotReachHere(); } ! // Line 3: chunk origin ! const ChunkOrigin origin = chunk->get_origin(); ! lines[2][pos] = origin == origin_normal ? ' ' : '0' + (int) origin; ! ! // Line 4: Virgin chunk? Virgin chunks are chunks created as a byproduct of padding or splitting, ! // but were never used. ! lines[3][pos] = chunk->get_use_count() > 0 ? ' ' : 'v'; ! ! p += spec_chunk_size; ! pos ++; ! } ! if (pos > 0) { ! for (int i = 0; i < NUM_LINES; i ++) { ! st->fill_to(22); ! st->print_raw(lines[i], line_len); ! st->cr(); ! } ! } ! for (int i = 0; i < NUM_LINES; i ++) { ! os::free(lines[i]); ! } ! } ! #ifdef ASSERT ! // Verify counters, all chunks in this list node and the occupancy map. ! void VirtualSpaceNode::verify(bool slow) { ! log_trace(gc, metaspace, freelist)("verifying %s virtual space node (%s).", ! (is_class() ? "class space" : "metaspace"), (slow ? "slow" : "quick")); ! // Fast mode: just verify chunk counters and basic geometry ! // Slow mode: verify chunks and occupancy map ! uintx num_in_use_chunks = 0; ! Metachunk* chunk = first_chunk(); ! Metachunk* invalid_chunk = (Metachunk*) top(); ! ! // Iterate the chunks in this node and verify each chunk. ! while (chunk < invalid_chunk ) { ! if (slow) { ! do_verify_chunk(chunk); ! } ! if (!chunk->is_tagged_free()) { ! num_in_use_chunks ++; ! } ! const size_t s = chunk->word_size(); ! // Prevent endless loop on invalid chunk size. ! assert(is_valid_chunksize(is_class(), s), "Invalid chunk size: " SIZE_FORMAT ".", s); ! MetaWord* next = ((MetaWord*)chunk) + s; ! chunk = (Metachunk*) next; ! } ! assert(_container_count == num_in_use_chunks, "Container count mismatch (real: " UINTX_FORMAT ! ", counter: " UINTX_FORMAT ".", num_in_use_chunks, _container_count); ! // Also verify the occupancy map. ! if (slow) { ! occupancy_map()->verify(bottom(), top()); ! } ! } ! // Verify that all free chunks in this node are ideally merged ! // (there not should be multiple small chunks where a large chunk could exist.) ! void VirtualSpaceNode::verify_free_chunks_are_ideally_merged() { ! Metachunk* chunk = first_chunk(); ! Metachunk* invalid_chunk = (Metachunk*) top(); ! // Shorthands. ! const size_t size_med = (is_class() ? ClassMediumChunk : MediumChunk) * BytesPerWord; ! const size_t size_small = (is_class() ? ClassSmallChunk : SmallChunk) * BytesPerWord; ! int num_free_chunks_since_last_med_boundary = -1; ! int num_free_chunks_since_last_small_boundary = -1; ! bool error = false; ! char err[256]; ! while (!error && chunk < invalid_chunk ) { ! // Test for missed chunk merge opportunities: count number of free chunks since last chunk boundary. ! // Reset the counter when encountering a non-free chunk. ! if (chunk->get_chunk_type() != HumongousIndex) { ! if (chunk->is_tagged_free()) { ! // Count successive free, non-humongous chunks. ! if (is_aligned(chunk, size_small)) { ! if (num_free_chunks_since_last_small_boundary > 0) { ! error = true; ! jio_snprintf(err, sizeof(err), "Missed chunk merge opportunity to merge a small chunk preceding " PTR_FORMAT ".", p2i(chunk)); ! } else { ! num_free_chunks_since_last_small_boundary = 0; ! } ! } else if (num_free_chunks_since_last_small_boundary != -1) { ! num_free_chunks_since_last_small_boundary ++; ! } ! if (is_aligned(chunk, size_med)) { ! if (num_free_chunks_since_last_med_boundary > 0) { ! error = true; ! jio_snprintf(err, sizeof(err), "Missed chunk merge opportunity to merge a medium chunk preceding " PTR_FORMAT ".", p2i(chunk)); ! } else { ! num_free_chunks_since_last_med_boundary = 0; ! } ! } else if (num_free_chunks_since_last_med_boundary != -1) { ! num_free_chunks_since_last_med_boundary ++; ! } ! } else { ! // Encountering a non-free chunk, reset counters. ! num_free_chunks_since_last_med_boundary = -1; ! num_free_chunks_since_last_small_boundary = -1; ! } ! } else { ! // One cannot merge areas with a humongous chunk in the middle. Reset counters. ! num_free_chunks_since_last_med_boundary = -1; ! num_free_chunks_since_last_small_boundary = -1; } ! if (error) { ! print_map(tty, is_class()); ! fatal("%s", err); ! } - MetaWord* next = ((MetaWord*)chunk) + chunk->word_size(); - chunk = (Metachunk*) next; - } } - #endif // ASSERT ! void VirtualSpaceNode::inc_container_count() { ! assert_lock_strong(MetaspaceExpand_lock); ! _container_count++; ! } - void VirtualSpaceNode::dec_container_count() { assert_lock_strong(MetaspaceExpand_lock); ! _container_count--; ! } ! VirtualSpaceNode::~VirtualSpaceNode() { ! _rs.release(); ! if (_occupancy_map != NULL) { ! delete _occupancy_map; ! } ! #ifdef ASSERT ! size_t word_size = sizeof(*this) / BytesPerWord; ! Copy::fill_to_words((HeapWord*) this, word_size, 0xf1f1f1f1); ! #endif ! } ! size_t VirtualSpaceNode::used_words_in_vs() const { ! return pointer_delta(top(), bottom(), sizeof(MetaWord)); ! } - // Space committed in the VirtualSpace - size_t VirtualSpaceNode::capacity_words_in_vs() const { - return pointer_delta(end(), bottom(), sizeof(MetaWord)); } ! size_t VirtualSpaceNode::free_words_in_vs() const { ! return pointer_delta(end(), top(), sizeof(MetaWord)); ! } ! // Given an address larger than top(), allocate padding chunks until top is at the given address. ! void VirtualSpaceNode::allocate_padding_chunks_until_top_is_at(MetaWord* target_top) { ! assert(target_top > top(), "Sanity"); ! // Padding chunks are added to the freelist. ! ChunkManager* const chunk_manager = Metaspace::get_chunk_manager(is_class()); ! // shorthands ! const size_t spec_word_size = chunk_manager->specialized_chunk_word_size(); ! const size_t small_word_size = chunk_manager->small_chunk_word_size(); ! const size_t med_word_size = chunk_manager->medium_chunk_word_size(); ! while (top() < target_top) { ! // We could make this coding more generic, but right now we only deal with two possible chunk sizes ! // for padding chunks, so it is not worth it. ! size_t padding_chunk_word_size = small_word_size; ! if (is_aligned(top(), small_word_size * sizeof(MetaWord)) == false) { ! assert_is_aligned(top(), spec_word_size * sizeof(MetaWord)); // Should always hold true. ! padding_chunk_word_size = spec_word_size; ! } ! MetaWord* here = top(); ! assert_is_aligned(here, padding_chunk_word_size * sizeof(MetaWord)); ! inc_top(padding_chunk_word_size); ! ! // Create new padding chunk. ! ChunkIndex padding_chunk_type = get_chunk_type_by_size(padding_chunk_word_size, is_class()); ! assert(padding_chunk_type == SpecializedIndex || padding_chunk_type == SmallIndex, "sanity"); ! ! Metachunk* const padding_chunk = ! ::new (here) Metachunk(padding_chunk_type, is_class(), padding_chunk_word_size, this); ! assert(padding_chunk == (Metachunk*)here, "Sanity"); ! DEBUG_ONLY(padding_chunk->set_origin(origin_pad);) ! log_trace(gc, metaspace, freelist)("Created padding chunk in %s at " ! PTR_FORMAT ", size " SIZE_FORMAT_HEX ".", ! (is_class() ? "class space " : "metaspace"), ! p2i(padding_chunk), padding_chunk->word_size() * sizeof(MetaWord)); ! ! // Mark chunk start in occupancy map. ! occupancy_map()->set_chunk_starts_at_address((MetaWord*)padding_chunk, true); ! ! // Chunks are born as in-use (see MetaChunk ctor). So, before returning ! // the padding chunk to its chunk manager, mark it as in use (ChunkManager ! // will assert that). ! do_update_in_use_info_for_chunk(padding_chunk, true); ! ! // Return Chunk to freelist. ! inc_container_count(); ! chunk_manager->return_single_chunk(padding_chunk); ! // Please note: at this point, ChunkManager::return_single_chunk() ! // may already have merged the padding chunk with neighboring chunks, so ! // it may have vanished at this point. Do not reference the padding ! // chunk beyond this point. ! } ! assert(top() == target_top, "Sanity"); ! } // allocate_padding_chunks_until_top_is_at() ! // Allocates the chunk from the virtual space only. ! // This interface is also used internally for debugging. Not all ! // chunks removed here are necessarily used for allocation. ! Metachunk* VirtualSpaceNode::take_from_committed(size_t chunk_word_size) { ! // Non-humongous chunks are to be allocated aligned to their chunk ! // size. So, start addresses of medium chunks are aligned to medium ! // chunk size, those of small chunks to small chunk size and so ! // forth. This facilitates merging of free chunks and reduces ! // fragmentation. Chunk sizes are spec < small < medium, with each ! // larger chunk size being a multiple of the next smaller chunk ! // size. ! // Because of this alignment, me may need to create a number of padding ! // chunks. These chunks are created and added to the freelist. ! ! // The chunk manager to which we will give our padding chunks. ! ChunkManager* const chunk_manager = Metaspace::get_chunk_manager(is_class()); ! ! // shorthands ! const size_t spec_word_size = chunk_manager->specialized_chunk_word_size(); ! const size_t small_word_size = chunk_manager->small_chunk_word_size(); ! const size_t med_word_size = chunk_manager->medium_chunk_word_size(); ! ! assert(chunk_word_size == spec_word_size || chunk_word_size == small_word_size || ! chunk_word_size >= med_word_size, "Invalid chunk size requested."); ! ! // Chunk alignment (in bytes) == chunk size unless humongous. ! // Humongous chunks are aligned to the smallest chunk size (spec). ! const size_t required_chunk_alignment = (chunk_word_size > med_word_size ? ! spec_word_size : chunk_word_size) * sizeof(MetaWord); ! ! // Do we have enough space to create the requested chunk plus ! // any padding chunks needed? ! MetaWord* const next_aligned = ! static_cast<MetaWord*>(align_up(top(), required_chunk_alignment)); ! if (!is_available((next_aligned - top()) + chunk_word_size)) { ! return NULL; } ! // Before allocating the requested chunk, allocate padding chunks if necessary. ! // We only need to do this for small or medium chunks: specialized chunks are the ! // smallest size, hence always aligned. Homungous chunks are allocated unaligned ! // (implicitly, also aligned to smallest chunk size). ! if ((chunk_word_size == med_word_size || chunk_word_size == small_word_size) && next_aligned > top()) { ! log_trace(gc, metaspace, freelist)("Creating padding chunks in %s between %p and %p...", ! (is_class() ? "class space " : "metaspace"), ! top(), next_aligned); ! allocate_padding_chunks_until_top_is_at(next_aligned); ! // Now, top should be aligned correctly. ! assert_is_aligned(top(), required_chunk_alignment); ! } ! // Now, top should be aligned correctly. ! assert_is_aligned(top(), required_chunk_alignment); ! // Bottom of the new chunk ! MetaWord* chunk_limit = top(); ! assert(chunk_limit != NULL, "Not safe to call this method"); ! ! // The virtual spaces are always expanded by the ! // commit granularity to enforce the following condition. ! // Without this the is_available check will not work correctly. ! assert(_virtual_space.committed_size() == _virtual_space.actual_committed_size(), ! "The committed memory doesn't match the expanded memory."); ! ! if (!is_available(chunk_word_size)) { ! LogTarget(Trace, gc, metaspace, freelist) lt; ! if (lt.is_enabled()) { ! LogStream ls(lt); ! ls.print("VirtualSpaceNode::take_from_committed() not available " SIZE_FORMAT " words ", chunk_word_size); ! // Dump some information about the virtual space that is nearly full ! print_on(&ls); ! } ! return NULL; ! } ! // Take the space (bump top on the current virtual space). ! inc_top(chunk_word_size); ! // Initialize the chunk ! ChunkIndex chunk_type = get_chunk_type_by_size(chunk_word_size, is_class()); ! Metachunk* result = ::new (chunk_limit) Metachunk(chunk_type, is_class(), chunk_word_size, this); ! assert(result == (Metachunk*)chunk_limit, "Sanity"); ! occupancy_map()->set_chunk_starts_at_address((MetaWord*)result, true); ! do_update_in_use_info_for_chunk(result, true); ! inc_container_count(); - #ifdef ASSERT - EVERY_NTH(VerifyMetaspaceInterval) - chunk_manager->locked_verify(true); - verify(true); - END_EVERY_NTH - do_verify_chunk(result); - #endif ! result->inc_use_count(); - return result; } - // Expand the virtual space (commit more of the reserved space) - bool VirtualSpaceNode::expand_by(size_t min_words, size_t preferred_words) { - size_t min_bytes = min_words * BytesPerWord; - size_t preferred_bytes = preferred_words * BytesPerWord; ! size_t uncommitted = virtual_space()->reserved_size() - virtual_space()->actual_committed_size(); ! if (uncommitted < min_bytes) { ! return false; ! } ! size_t commit = MIN2(preferred_bytes, uncommitted); ! bool result = virtual_space()->expand_by(commit, false); - if (result) { - log_trace(gc, metaspace, freelist)("Expanded %s virtual space list node by " SIZE_FORMAT " words.", - (is_class() ? "class" : "non-class"), commit); - DEBUG_ONLY(Atomic::inc(&g_internal_statistics.num_committed_space_expanded)); - } else { - log_trace(gc, metaspace, freelist)("Failed to expand %s virtual space list node by " SIZE_FORMAT " words.", - (is_class() ? "class" : "non-class"), commit); } ! assert(result, "Failed to commit memory"); - return result; } ! Metachunk* VirtualSpaceNode::get_chunk_vs(size_t chunk_word_size) { assert_lock_strong(MetaspaceExpand_lock); - Metachunk* result = take_from_committed(chunk_word_size); - return result; - } ! bool VirtualSpaceNode::initialize() { ! if (!_rs.is_reserved()) { ! return false; ! } ! // These are necessary restriction to make sure that the virtual space always ! // grows in steps of Metaspace::commit_alignment(). If both base and size are ! // aligned only the middle alignment of the VirtualSpace is used. ! assert_is_aligned(_rs.base(), Metaspace::commit_alignment()); ! assert_is_aligned(_rs.size(), Metaspace::commit_alignment()); ! ! // ReservedSpaces marked as special will have the entire memory ! // pre-committed. Setting a committed size will make sure that ! // committed_size and actual_committed_size agrees. ! size_t pre_committed_size = _rs.special() ? _rs.size() : 0; ! ! bool result = virtual_space()->initialize_with_granularity(_rs, pre_committed_size, ! Metaspace::commit_alignment()); ! if (result) { ! assert(virtual_space()->committed_size() == virtual_space()->actual_committed_size(), ! "Checking that the pre-committed memory was registered by the VirtualSpace"); ! set_top((MetaWord*)virtual_space()->low()); ! } ! // Initialize Occupancy Map. ! const size_t smallest_chunk_size = is_class() ? ClassSpecializedChunk : SpecializedChunk; ! _occupancy_map = new OccupancyMap(bottom(), reserved_words(), smallest_chunk_size); - return result; } void VirtualSpaceNode::print_on(outputStream* st, size_t scale) const { - size_t used_words = used_words_in_vs(); - size_t commit_words = committed_words(); - size_t res_words = reserved_words(); - VirtualSpace* vs = virtual_space(); st->print("node @" PTR_FORMAT ": ", p2i(this)); st->print("reserved="); ! print_scaled_words(st, res_words, scale); st->print(", committed="); ! print_scaled_words_and_percentage(st, commit_words, res_words, scale); st->print(", used="); ! print_scaled_words_and_percentage(st, used_words, res_words, scale); st->cr(); ! st->print(" [" PTR_FORMAT ", " PTR_FORMAT ", " ! PTR_FORMAT ", " PTR_FORMAT ")", ! p2i(bottom()), p2i(top()), p2i(end()), ! p2i(vs->high_boundary())); } #ifdef ASSERT ! void VirtualSpaceNode::mangle() { ! size_t word_size = capacity_words_in_vs(); ! Copy::fill_to_words((HeapWord*) low(), word_size, 0xf1f1f1f1); } - #endif // ASSERT ! void VirtualSpaceNode::retire(ChunkManager* chunk_manager) { ! assert(is_class() == chunk_manager->is_class(), "Wrong ChunkManager?"); ! #ifdef ASSERT ! verify(false); ! EVERY_NTH(VerifyMetaspaceInterval) ! verify(true); ! END_EVERY_NTH ! #endif ! for (int i = (int)MediumIndex; i >= (int)ZeroIndex; --i) { ! ChunkIndex index = (ChunkIndex)i; ! size_t chunk_size = chunk_manager->size_by_index(index); ! ! while (free_words_in_vs() >= chunk_size) { ! Metachunk* chunk = get_chunk_vs(chunk_size); ! // Chunk will be allocated aligned, so allocation may require ! // additional padding chunks. That may cause above allocation to ! // fail. Just ignore the failed allocation and continue with the ! // next smaller chunk size. As the VirtualSpaceNode comitted ! // size should be a multiple of the smallest chunk size, we ! // should always be able to fill the VirtualSpace completely. ! if (chunk == NULL) { ! break; ! } ! chunk_manager->return_single_chunk(chunk); ! } ! } ! assert(free_words_in_vs() == 0, "should be empty now"); } } // namespace metaspace --- 20,369 ---- * or visit www.oracle.com if you need additional information or have any * questions. * */ + #include "precompiled.hpp" #include "logging/log.hpp" ! ! #include "memory/metaspace/chunkLevel.hpp" ! #include "memory/metaspace/chunkTree.hpp" ! #include "memory/metaspace/commitLimiter.hpp" ! #include "memory/metaspace/constants.hpp" ! #include "memory/metaspace/counter.hpp" #include "memory/metaspace/metachunk.hpp" #include "memory/metaspace/metaspaceCommon.hpp" ! #include "memory/metaspace/runningCounters.hpp" #include "memory/metaspace/virtualSpaceNode.hpp" ! ! #include "runtime/mutexLocker.hpp" #include "runtime/os.hpp" ! ! #include "utilities/align.hpp" #include "utilities/debug.hpp" #include "utilities/globalDefinitions.hpp" + #include "utilities/ostream.hpp" namespace metaspace { ! #ifdef ASSERT ! template <class T> ! void check_is_aligned_to_commit_granule(T x) { ! assert(is_aligned(x, constants::commit_granule_bytes), "Unaligned pointer"); } + #endif ! // Given an address range, ensure it is committed. ! // ! // The range has to be aligned to granule size. ! // ! // Function will: ! // - check how many granules in that region are uncommitted; If all are committed, it ! // returns true immediately. ! // - check if committing those uncommitted granules would bring us over the commit limit ! // (GC threshold, MaxMetaspaceSize). If true, it returns false. ! // - commit the memory. ! // - mark the range as committed in the commit mask ! // ! // Returns true if success, false if it did hit a commit limit. ! bool VirtualSpaceNode::commit_range(MetaWord* p, size_t word_size) { ! DEBUG_ONLY(check_is_aligned_to_commit_granule(p);) ! DEBUG_ONLY(check_is_aligned_to_commit_granule(word_size);) ! assert_lock_strong(MetaspaceExpand_lock); ! // First calculate how large the committed regions in this range are ! const size_t committed_words_in_range = _commit_mask.get_committed_size_in_range(p, word_size); ! DEBUG_ONLY(check_is_aligned_to_commit_granule(committed_words_in_range);) ! // By how much words we would increase commit charge ! // were we to commit the given address range completely. ! const size_t commit_increase_words = word_size - committed_words_in_range; ! if (commit_increase_words == 0) { ! return true; // Already fully committed, nothing to do. } ! // Before committing any more memory, check limits. ! if (_commit_limiter->possible_expansion_words() < commit_increase_words) { ! return false; } ! // Commit... ! if (os::commit_memory((char*)p, word_size * BytesPerWord, false) == false) { ! vm_exit_out_of_memory(word_size * BytesPerWord, OOM_MMAP_ERROR, "Failed to commit metaspace."); } ! log_debug(gc, metaspace)("Increased metaspace by " SIZE_FORMAT " bytes.", ! commit_increase_words * BytesPerWord); + // ... tell commit limiter... + _commit_limiter->increase_committed(commit_increase_words); ! // ... update counters in containing vslist ... ! _total_committed_words_counter->increment_by(commit_increase_words); ! // ... and update the commit mask. ! _commit_mask.mark_range_as_committed(p, word_size); ! #ifdef ASSERT ! // The commit boundary maintained in the CommitLimiter should be equal the sum of committed words ! // in both class and non-class vslist (outside gtests). ! if (_commit_limiter == CommitLimiter::globalLimiter()) { ! assert(_commit_limiter->committed_words() == RunningCounters::committed_words(), "counter mismatch"); } + #endif ! return true; } ! // Given an address range, ensure it is committed. ! // ! // The range does not have to be aligned to granule size. However, the function will always commit ! // whole granules. ! // ! // Function will: ! // - check how many granules in that region are uncommitted; If all are committed, it ! // returns true immediately. ! // - check if committing those uncommitted granules would bring us over the commit limit ! // (GC threshold, MaxMetaspaceSize). If true, it returns false. ! // - commit the memory. ! // - mark the range as committed in the commit mask ! // ! // Returns true if success, false if it did hit a commit limit. ! bool VirtualSpaceNode::ensure_range_is_committed(MetaWord* p, size_t word_size) { assert_lock_strong(MetaspaceExpand_lock); ! assert(p != NULL && word_size > 0, "Sanity"); ! MetaWord* p_start = align_down(p, constants::commit_granule_bytes); ! MetaWord* p_end = align_up(p + word_size, constants::commit_granule_bytes); ! // Todo: simple for now. Make it more intelligent late ! return commit_range(p_start, p_end - p_start); } ! // Given an address range (which has to be aligned to commit granule size): ! // - uncommit it ! // - mark it as uncommitted in the commit mask ! bool VirtualSpaceNode::uncommit_range(MetaWord* p, size_t word_size) { ! DEBUG_ONLY(check_is_aligned_to_commit_granule(p);) ! DEBUG_ONLY(check_is_aligned_to_commit_granule(word_size);) ! assert_lock_strong(MetaspaceExpand_lock); ! // First calculate how large the committed regions in this range are ! const size_t committed_words_in_range = _commit_mask.get_committed_size_in_range(p, word_size); ! DEBUG_ONLY(check_is_aligned_to_commit_granule(committed_words_in_range);) ! if (committed_words_in_range == 0) { ! return true; // Already fully uncommitted, nothing to do. ! } ! // Uncommit... ! if (os::uncommit_memory((char*)p, word_size * BytesPerWord) == false) { ! // Note: this can actually happen, since uncommit may increase the number of mappings. ! fatal("Failed to uncommit metaspace."); ! } ! log_debug(gc, metaspace)("Decreased metaspace by " SIZE_FORMAT " bytes.", ! committed_words_in_range * BytesPerWord); ! // ... tell commit limiter... ! _commit_limiter->decrease_committed(committed_words_in_range); ! // ... and global counters... ! _total_committed_words_counter->decrement_by(committed_words_in_range); ! // ... and update the commit mask. ! _commit_mask.mark_range_as_uncommitted(p, word_size); ! #ifdef ASSERT ! // The commit boundary maintained in the CommitLimiter should be equal the sum of committed words ! // in both class and non-class vslist (outside gtests). ! if (_commit_limiter == CommitLimiter::globalLimiter()) { // We are outside a test scenario ! assert(_commit_limiter->committed_words() == RunningCounters::committed_words(), "counter mismatch"); } + #endif ! return true; ! } ! //// creation, destruction //// ! ! VirtualSpaceNode::VirtualSpaceNode(ReservedSpace rs, ! CommitLimiter* limiter, ! SizeCounter* reserve_counter, ! SizeCounter* commit_counter) ! : _next(NULL), ! _base((MetaWord*)rs.base()), ! _word_size(rs.size() / BytesPerWord), ! _used_words(0), ! _commit_mask((MetaWord*)rs.base(), rs.size() / BytesPerWord), ! _chunk_tree_array((MetaWord*)rs.base(), rs.size() / BytesPerWord), ! _commit_limiter(limiter), ! _total_reserved_words_counter(reserve_counter), ! _total_committed_words_counter(commit_counter) ! { ! // Update reserved counter in vslist ! _total_reserved_words_counter->increment_by(_word_size); ! } ! ! // Create a node of a given size ! VirtualSpaceNode* VirtualSpaceNode::create_node(size_t word_size, ! CommitLimiter* limiter, ! SizeCounter* reserve_counter, ! SizeCounter* commit_counter) ! { ! ! DEBUG_ONLY(check_is_aligned_to_commit_granule(word_size);) ! ! ReservedSpace rs(word_size * BytesPerWord, ! constants::commit_granule_bytes, ! false, // TODO deal with large pages ! false); ! ! if (!rs.is_reserved()) { ! vm_exit_out_of_memory(word_size * BytesPerWord, OOM_MMAP_ERROR, "Failed to reserve memory for metaspace"); ! } ! ! reserve_counter->increment_by(word_size * BytesPerWord); ! ! return create_node(rs, limiter, reserve_counter, commit_counter); ! } ! // Create a node over an existing space ! VirtualSpaceNode* VirtualSpaceNode::create_node(ReservedSpace rs, ! CommitLimiter* limiter, ! SizeCounter* reserve_counter, ! SizeCounter* commit_counter) ! { ! reserve_counter->increment_by(rs.size() * BytesPerWord); ! return new VirtualSpaceNode(rs, limiter, reserve_counter, commit_counter); ! } ! VirtualSpaceNode::~VirtualSpaceNode() { ! _rs.release(); ! // Update counters in vslist ! _total_committed_words_counter->decrement_by(committed_words()); ! _total_reserved_words_counter->decrement_by(_word_size); } ! //// Chunk allocation, splitting, merging ///// ! // Allocate a root chunk from this node. Will fail and return NULL ! // if the node is full. ! // Note: this just returns a chunk whose memory is reserved; no memory is committed yet. ! // Hence, before using this chunk, it must be committed. ! // Also, no limits are checked, since no committing takes place. ! Metachunk* VirtualSpaceNode::allocate_root_chunk() { ! ! assert_lock_strong(MetaspaceExpand_lock); ! ! assert_is_aligned(free_words(), chklvl::MAX_CHUNK_WORD_SIZE); ! ! if (free_words() >= chklvl::MAX_CHUNK_WORD_SIZE) { ! ! MetaWord* loc = _base + _used_words; ! _used_words += chklvl::MAX_CHUNK_WORD_SIZE; ! ! // Create a new chunk tree for that new root node. ! ChunkTree* tree = _chunk_tree_array.get_tree_by_address(loc); ! ! // Create a root chunk header and initialize it; ! Metachunk* c = tree->alloc_root_chunk_header(); ! c->set_base(loc); ! c->set_vsnode(this); ! c->set_free(true); ! ! DEBUG_ONLY(c->verify(true);) ! return c; } ! return NULL; // Node is full. } ! Metachunk* VirtualSpaceNode::split(chklvl_t target_level, Metachunk* c, Metachunk* splinters[chklvl::NUM_CHUNK_LEVELS]) { ! assert_lock_strong(MetaspaceExpand_lock); ! // Get the tree associated with this chunk and let it handle the splitting ! ChunkTree* tree = _chunk_tree_array.get_tree_by_address(c->base()); ! return tree->split(target_level, c, splinters); ! } ! Metachunk* VirtualSpaceNode::merge(Metachunk* c, int num_merged[chklvl::NUM_CHUNK_LEVELS]) { ! assert_lock_strong(MetaspaceExpand_lock); ! // Get the tree associated with this chunk and let it handle the merging ! ChunkTree* tree = _chunk_tree_array.get_tree_by_address(c->base()); ! return tree->merge(c, num_merged); } void VirtualSpaceNode::print_on(outputStream* st, size_t scale) const { st->print("node @" PTR_FORMAT ": ", p2i(this)); st->print("reserved="); ! print_scaled_words(st, word_size(), scale); st->print(", committed="); ! print_scaled_words_and_percentage(st, committed_words(), word_size(), scale); st->print(", used="); ! print_scaled_words_and_percentage(st, used_words(), word_size(), scale); ! st->cr(); ! ! st->print_cr(" [" PTR_FORMAT ", " PTR_FORMAT ", " PTR_FORMAT ")", ! p2i(base()), p2i(base() + used_words()), p2i(base() + word_size())); ! ! st->print("Tree/Chunk footprint: "); ! print_scaled_words(st, _chunk_tree_array.memory_footprint_words(), scale); ! ! st->cr(); ! } + #ifdef ASSERT ! // Verify counters and basic structure. Slow mode: verify all chunks in depth ! void VirtualSpaceNode::verify(bool slow) const { ! ! assert_lock_strong(MetaspaceExpand_lock); ! ! assert(base() != NULL, "Invalid base"); ! assert_is_aligned(base(), chklvl::MAX_CHUNK_BYTE_SIZE); ! assert(used_words() < word_size(), "Sanity"); ! ! // Since we only ever hand out root chunks from a vsnode, top should always be aligned ! // to root chunk size. ! assert_is_aligned(used_words(), chklvl::MAX_CHUNK_WORD_SIZE); ! ! _commit_mask.verify(slow); ! _chunk_tree_array.verify(slow); ! } ! // Returns sum of committed space, in words. ! size_t VirtualSpaceNode::committed_words() const { ! return _commit_mask.get_committed_size(); } + #endif + } // namespace metaspace
< prev index next >