提交 7e111f60 编写于 作者: M maoliang.ml 提交者: mmyxym

[GC] Port G1ElasticHeap to dragonwell

Summary: G1ElasticHeap to support heap shrink/expansion in runtime

Test Plan:
test/elastic-heap/

Reviewers:

Issue:
https://github.com/alibaba/dragonwell8/issues/46

CR:
https://github.com/alibaba/dragonwell8_hotspot/pull/12
上级 b0fb410e
......@@ -55,6 +55,8 @@ public enum GCCause {
_g1_inc_collection_pause ("G1 Evacuation Pause"),
_g1_humongous_allocation ("G1 Humongous Allocation"),
_g1_elastic_heap_trigger_gc("Elastic Heap triggered GC"),
_last_ditch_collection ("Last ditch collection"),
_last_gc_cause ("ILLEGAL VALUE - last gc cause - ILLEGAL VALUE");
......
......@@ -84,6 +84,15 @@ SUNWprivate_1.1 {
JVM_DumpAllStacks;
JVM_DumpThreads;
JVM_EnableCompiler;
JVM_ElasticHeapGetEvaluationMode;
JVM_ElasticHeapSetYoungGenCommitPercent;
JVM_ElasticHeapGetYoungGenCommitPercent;
JVM_ElasticHeapSetUncommitIHOP;
JVM_ElasticHeapGetUncommitIHOP;
JVM_ElasticHeapGetTotalYoungUncommittedBytes;
JVM_ElasticHeapSetSoftmxPercent;
JVM_ElasticHeapGetSoftmxPercent;
JVM_ElasticHeapGetTotalUncommittedBytes;
JVM_Exit;
JVM_FillInStackTrace;
JVM_FindClassFromCaller;
......
......@@ -69,6 +69,15 @@ SUNWprivate_1.1 {
JVM_CurrentClassLoader;
JVM_CurrentLoadedClass;
JVM_CurrentThread;
JVM_ElasticHeapGetEvaluationMode;
JVM_ElasticHeapSetYoungGenCommitPercent;
JVM_ElasticHeapGetYoungGenCommitPercent;
JVM_ElasticHeapSetUncommitIHOP;
JVM_ElasticHeapGetUncommitIHOP;
JVM_ElasticHeapGetTotalYoungUncommittedBytes;
JVM_ElasticHeapSetSoftmxPercent;
JVM_ElasticHeapGetSoftmxPercent;
JVM_ElasticHeapGetTotalUncommittedBytes;
JVM_CurrentTimeMillis;
JVM_DefineClass;
JVM_DefineClassWithSource;
......
此差异已折叠。
/*
* Copyright (c) 2019 Alibaba Group Holding Limited. All Rights Reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Alibaba designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*/
#ifndef SHARE_VM_GC_IMPLEMENTATION_G1_ELASTICHEAP_HPP
#define SHARE_VM_GC_IMPLEMENTATION_G1_ELASTICHEAP_HPP
#include "gc_implementation/g1/heapRegionSet.hpp"
#include "gc_implementation/g1/heapRegion.hpp"
#include "runtime/mutex.hpp"
#include "runtime/thread.inline.hpp"
#include "utilities/exceptions.hpp"
#define GC_INTERVAL_SEQ_LENGTH 10
class ElasticHeap;
// ElasticHeapConcThread:
// Commit/uncommit memory (several GB) may cost significant time so we won't do it in STW.
// We use a concurrent thread for doing commit/uncommit/pretouch the memory of regions
// Heap regions will be removed from free list of _hrm and put into commit/uncommit list
// for doing the commit/uncommit job and moved back into _hrm in next GC stw
// after commit/uncommit finishes.
class ElasticHeapConcThread : public NamedThread {
friend class ElasticHeap;
private:
ElasticHeap* _elastic_heap;
// List of regions to uncommit
FreeRegionList _uncommit_list;
// List of regions to commit
FreeRegionList _commit_list;
// List of old regions to free
FreeRegionList _to_free_list;
// Indicate the commit/uncommit is working
// Will be set when VM thread(GC) triggers ElasticHeapThread to do memory job
// and be cleared when the memory job finishes
//
// Java threads(Jcmd listener or MXBean) will check this flag if the memory work is done
// ElasticHeap::recover_for_full_gc will also check this flag
volatile bool _working;
// ElasticHeapThread will wait on it for doing memory job
// And VM trhead(GC) will notify ElasticHeapThread to do memory job via _conc_lock
Monitor* _conc_lock;
// Java threads(Jcmd listener or MXBean) will wait on _working_lock for the finish of memory work
// ElasticHeap::recover_for_full_gc will also wait on _working_lock for the finish of memory work
Monitor* _working_lock;
// Number of parallel worker threads
uint _parallel_worker_threads;
// Parallel workers
FlexibleWorkGang* _parallel_workers;
// For terminating conc thread
bool _should_terminate;
bool _has_terminated;
void sanity_check();
void print_work_summary(uint uncommit_length, uint commit_length, double start);
// Commit/Uncommit phyical pages of regions
void do_memory_job();
// Commit/Uncommit phyical pages of regions
uint do_memory(FreeRegionList* list, HeapRegionClosure* cl);
// Parallel Commit/Uncommit phyical pages
void par_work_on_regions(FreeRegionList* list, HeapRegionClosure* cl);
void wait_for_universe_init();
public:
ElasticHeapConcThread(ElasticHeap* elastic_heap);
// Region list for commit/uncommit
FreeRegionList* uncommit_list() { return &_uncommit_list; }
FreeRegionList* commit_list() { return &_commit_list; }
FreeRegionList* to_free_list() { return &_to_free_list; }
void start();
bool working() const { return _working; }
void set_working();
void clear_working() {
assert(_working, "Sanity");
_working = false;
}
// Lock for manipulating ElasticHeapThread data structures
Monitor* conc_lock() { return _conc_lock; }
Monitor* working_lock() { return _working_lock; }
void sleep_before_next_cycle();
virtual void run();
void stop();
// Count of regions in FreeRegionList
uint total_unavaiable_regions();
};
// Data structure for elastic heap timer thread
// We will trigger young gc in this thread
// but we cannot do it in a regular native thread(a limitation in JDK1.8)
// so we have to make it a Java thread
class ElasticHeapTimer : AllStatic {
friend class ElasticHeap;
private:
volatile static bool _should_terminate;
static JavaThread* _thread;
static Monitor* _monitor;
public:
// Timer thread entry
static void timer_thread_entry(JavaThread* thread, TRAPS);
static void start();
static void stop();
static bool has_error(TRAPS, const char* error);
};
class ElasticHeapEvaluator : public CHeapObj<mtGC> {
protected:
ElasticHeap* _elas;
G1CollectedHeap* _g1h;
G1CollectorPolicy* _g1_policy;
HeapRegionManager* _hrm;
public:
ElasticHeapEvaluator(ElasticHeap* eh);
void sanity_check();
virtual void evaluate() = 0;
virtual void evaluate_young() { ShouldNotReachHere(); }
virtual void evaluate_old() { ShouldNotReachHere(); }
virtual void evaluate_old_common();
virtual bool ready_to_initial_mark() { ShouldNotReachHere(); }
};
class RecoverEvaluator : public ElasticHeapEvaluator {
public:
RecoverEvaluator(ElasticHeap* eh)
: ElasticHeapEvaluator(eh) {}
virtual void evaluate();
};
class PeriodicEvaluator : public ElasticHeapEvaluator {
public:
PeriodicEvaluator(ElasticHeap* eh)
: ElasticHeapEvaluator(eh) {}
virtual void evaluate();
virtual void evaluate_young();
virtual void evaluate_old() { evaluate_old_common(); }
virtual bool ready_to_initial_mark();
};
class GenerationLimitEvaluator : public ElasticHeapEvaluator {
public:
GenerationLimitEvaluator(ElasticHeap* eh)
: ElasticHeapEvaluator(eh) {}
virtual void evaluate();
virtual void evaluate_young();
virtual void evaluate_old() { evaluate_old_common(); }
virtual bool ready_to_initial_mark();
};
class SoftmxEvaluator : public ElasticHeapEvaluator {
public:
SoftmxEvaluator(ElasticHeap* eh)
: ElasticHeapEvaluator(eh) {}
virtual void evaluate();
};
class ElasticHeapGCStats : public CHeapObj<mtGC> {
friend class ElasticHeap;
public:
ElasticHeapGCStats(ElasticHeap*, G1CollectedHeap*);
double avg_normalized_interval() const {
return _recent_ygc_normalized_interval_ms->avg();
}
double last_normalized_interval() const {
return _recent_ygc_normalized_interval_ms->last();
}
int num_normalized_interval() const {
return _recent_ygc_normalized_interval_ms->num();
}
uint last_normalized_eden_consumed_length() const {
return _last_normalized_eden_consumed_length;
}
double last_initial_mark_timestamp_s() const {
return _last_initial_mark_timestamp_s;
}
double last_initial_mark_interval_s() const {
return _last_initial_mark_interval_s;
}
size_t last_initial_mark_non_young_bytes() const {
return _last_initial_mark_non_young_bytes;
}
// Reset the tracking info of mixed gc
void reset_mixed_gc_info() {
_last_gc_mixed = false;
_mixed_gc_finished = false;;
_need_mixed_gc = false;
}
bool last_gc_mixed() const { return _last_gc_mixed; }
bool need_mixed_gc() const { return _need_mixed_gc; }
void set_need_mixed_gc(bool f) { _need_mixed_gc = f; }
double last_gc_end_timestamp_s() const { return _last_gc_end_timestamp_s; }
void set_last_gc_end_timestamp_s(double s) { _last_gc_end_timestamp_s = s; }
void track_gc_start(bool full_gc);
bool check_mixed_gc_finished();
private:
ElasticHeap* _elas;
G1CollectedHeap* _g1h;
// G1 will use non-fixed size of young generation
// We scale the gc interval to a normalized interval as max size of young generation
// e.g. we make interval X 2 if last GC only used 1/2 of max size of young gen
TruncatedSeq* _recent_ygc_normalized_interval_ms;
// Time of last GC in seccond
volatile double _last_gc_end_timestamp_s;
// Time of initial mark in seccond
volatile double _last_initial_mark_timestamp_s;
// If last gc is mixed gc
volatile bool _last_gc_mixed;
// If mixed gc has finished
volatile bool _mixed_gc_finished;
// If mixed gc needs to happen
volatile bool _need_mixed_gc;
// The interval in second between last 2 intial-marks
volatile double _last_initial_mark_interval_s;
// Bytes of non-young part in last initial mark
size_t _last_initial_mark_non_young_bytes;
// Number of eden regions consumed in last gc interval(normalized)
uint _last_normalized_eden_consumed_length;
};
class ElasticHeapSetting;
// ElasticHeap:
// Main class of elastic heap feature
class ElasticHeap : public CHeapObj<mtGC> {
friend class ElasticHeapEvaluator;
friend class RecoverEvaluator;
friend class PeriodicEvaluator;
friend class GenerationLimitEvaluator;
friend class SoftmxEvaluator;
public:
ElasticHeap(G1CollectedHeap* g1h);
~ElasticHeap();
enum ErrorType {
NoError = 0,
HeapAdjustInProgress,
GCTooFrequent,
IllegalYoungPercent,
IllegalMode
};
// Modes when elastic heap is activated
enum EvaluationMode {
InactiveMode = 0,
PeriodicUncommitMode,
GenerationLimitMode,
SoftmxMode,
EvaluationModeNum
};
static uint ignore_arg();
ElasticHeapGCStats* stats() { return _stats; }
ElasticHeapSetting* setting() { return _setting; }
private:
G1CollectedHeap* _g1h;
ElasticHeapGCStats* _stats;
ElasticHeapSetting* _setting;
// Initial value of InitiatingHeapOccupancyPercent
uint _orig_ihop;
// Initial value of desired young length
uint _orig_min_desired_young_length;
uint _orig_max_desired_young_length;
// Indicates in the concurrent working cycle of memory job
// 1. A concurrent cycle starts (start_conc_cycle) in VM Thread(GC stw) when triggering
// ElasticHeapConcThread to do memory job
// 2. A concurrent cycle ends (end_conc_cycle) in the next gc pause after region
// commit/uncommit finishes and move regions in ElasticHeapConcThread back into _hrm
volatile bool _in_conc_cycle;
// Concurrent thread for doing memory job
ElasticHeapConcThread* _conc_thread;
volatile int _configure_setting_lock;
ElasticHeapEvaluator* _evaluators[EvaluationModeNum];
bool _heap_capacity_changed;
void start_conc_cycle();
void end_conc_cycle();
void move_regions_back_to_hrm();
// Move the regions in _thread back to _hrm and clear in_cycle
void finish_conc_cycle();
// Check completion of previous ElasticHeapConcThread work
void check_end_of_previous_conc_cycle();
// Update heap size information after expanding completes
void update_expanded_heap_size();
// Do the elastic heap evaluation and operation
void evaluate_elastic_work();
void try_starting_conc_cycle();
void wait_for_conc_thread_working_done(bool safepointCheck);
bool enough_free_regions_to_uncommit(uint num_regions_to_uncommit);
// Update min/max desired young length after change young length
void update_desired_young_length(uint unavailable_young_length);
public:
void uncommit_region_memory(HeapRegion* hr);
void commit_region_memory(HeapRegion* hr, bool pretouch);
void free_region_memory(HeapRegion* hr);
bool in_conc_cycle() { return _in_conc_cycle; }
// Record gc information
void record_gc_start(bool full_gc = false);
void record_gc_end();
void prepare_in_gc_start();
// Main entry in GC pause for elastic heap
void perform();
// Commit/uncommit regions
void uncommit_regions(uint num);
void commit_regions(uint num);
void resize_young_length(uint target_length);
void change_heap_capacity(uint target_heap_regions);
void change_young_size_for_softmx();
uint max_young_length() const { return _orig_max_desired_young_length; }
// Number of regions that old gen overlap with young gen
// So young gen cannot use as many as MaxNewSize
uint overlapped_young_regions_with_old_gen();
uint num_unavailable_regions();
// Main entry for processing elastic heap via JCMD/MXBean
ErrorType configure_setting(uint young_percent, uint uncommit_ihop, uint softmx_percent);
// Get the commit percent and freed bytes
int young_commit_percent() const;
jlong young_uncommitted_bytes() const;
int softmx_percent() const;
jlong uncommitted_bytes() const;
int uncommit_ihop() const;
// Wait for the conc thread working done and finish the current cycle
void wait_for_conc_cycle_end();
// If Full GC happens in auto mode, elastic heap will wait to recover the uncommitted regions
void wait_to_recover();
// If the gc is triggered by elastic heap to resize young gen
bool is_gc_to_resize_young_gen();
// Max survivor regions in this gc.
// We probably do a young gen resize in the end of gc
// so we cannot leave a too large survior size if we will make young gen smaller
uint max_available_survivor_regions();
// Check to initiate initial-mark
void check_to_initate_conc_mark();
// Check to invoke a GC
void check_to_trigger_ygc();
// With softmx mode, we will change the capacity of heap
bool heap_capacity_changed() { return _heap_capacity_changed; }
void set_heap_capacity_changed(uint num);
EvaluationMode evaluation_mode() const;
bool conflict_mode(EvaluationMode target_mode);
// Get string from error type
static const char* to_string(ErrorType type) {
switch(type) {
case HeapAdjustInProgress: return "last elastic-heap resize is still in progress";
case GCTooFrequent: return "gc is too frequent";
case IllegalYoungPercent: return "illegal percent";
case IllegalMode: return "not in correct mode";
default: ShouldNotReachHere(); return NULL;
}
}
// Get string from EvaluationMode
static const char* to_string(EvaluationMode mode) {
switch (mode) {
case InactiveMode: return "inactive";
case PeriodicUncommitMode: return "periodic uncommit";
case GenerationLimitMode: return "generation limit";
case SoftmxMode: return "softmx";
default: ShouldNotReachHere(); return NULL;
}
}
void destroy();
bool can_turn_on_periodic_uncommit();
bool ready_to_uncommit_after_mixed_gc();
void uncommit_old_gen();
uint calculate_young_list_desired_max_length();
double g1_reserve_factor();
uint g1_reserve_regions();
};
class ElasticHeapSetting : public CHeapObj<mtGC> {
friend class ElasticHeap;
public:
ElasticHeapSetting(ElasticHeap*, G1CollectedHeap*);
// Whether do elastic heap resize by explicit command
bool generation_limit_set() const {
return young_percent_set() || uncommit_ihop_set();
}
// Whether a jcmd/mxbean command set the young commit percent
bool young_percent_set() const {
return _young_percent != 0;
}
uint young_percent() const {
return _young_percent;
}
uint set_young_percent(uint p) {
_young_percent = p;
}
// Whether a jcmd/mxbean command set the uncommit ihop
bool uncommit_ihop_set() const {
return _uncommit_ihop != 0;
}
uint uncommit_ihop() const {
return _uncommit_ihop;
}
bool softmx_percent_set() const {
return _softmx_percent != 0;
}
uint softmx_percent() const {
return _softmx_percent;
}
void set_softmx_percent(uint p) {
_softmx_percent = p;
}
ElasticHeap::ErrorType change_young_percent(uint young_percent, bool& trigger_gc);
ElasticHeap::ErrorType change_uncommit_ihop(uint uncommit_ihop, bool& trigger_gc);
ElasticHeap::ErrorType change_softmx_percent(uint softmx_percent, bool& trigger_gc);
ElasticHeap::ErrorType process_arg(uint young_percent, uint uncommit_ihop, uint softmx_percent,
bool& trigger_gc);
ElasticHeap::EvaluationMode target_evaluation_mode(uint young_percent,
uint uncommit_ihop,
uint softmx_percent);
private:
ElasticHeap* _elas;
G1CollectedHeap* _g1h;
// Young list percentage set by command
volatile uint _young_percent;
// Set the initiating heap occupancy for elastic heap and do uncommit after concurrent gc
volatile uint _uncommit_ihop;
// Set the softmx percent
volatile uint _softmx_percent;
// Check if we can change the young percent by jcmd/MXBean
bool can_change_young_percent(uint percent);
static uint ignore_arg() { return (uint)-1; }
static bool ignore_arg(uint arg, uint cur_value) {
return arg == cur_value || arg == ignore_arg();
}
};
#endif // SHARE_VM_GC_IMPLEMENTATION_G1_ELASTICHEAP_HPP
......@@ -67,6 +67,7 @@
#include "oops/oop.pcgc.inline.hpp"
#include "runtime/orderAccess.inline.hpp"
#include "runtime/vmThread.hpp"
#include "gc_implementation/g1/elasticHeap.hpp"
size_t G1CollectedHeap::_humongous_object_threshold_in_words = 0;
......@@ -775,7 +776,8 @@ HeapWord* G1CollectedHeap::humongous_obj_allocate(size_t word_size, AllocationCo
}
}
if (first == G1_NO_HRM_INDEX) {
// With G1ElasticHeap we don't allocate from (elastic) unavailable regions
if (first == G1_NO_HRM_INDEX && !G1ElasticHeap) {
// Policy: We could not find enough regions for the humongous object in the
// free list. Look through the heap to find a mix of free and uncommitted regions.
// If so, try expansion.
......@@ -1296,6 +1298,21 @@ bool G1CollectedHeap::do_collection(bool explicit_gc,
double start = os::elapsedTime();
g1_policy()->record_full_collection_start();
// With ElasticHeap, we may wait to finish some work
if (G1ElasticHeap) {
elastic_heap()->record_gc_start(true);
if (explicit_gc) {
// In explicit full gc, wait for conc cycle to finish
elastic_heap()->wait_for_conc_cycle_end();
} else {
if (ElasticHeapPeriodicUncommit) {
// If not explicit full gc and in elastic heap periodic GC mode
// recover the uncommitted regions
elastic_heap()->wait_to_recover();
}
}
}
// Note: When we have a more flexible GC logging framework that
// allows us to add optional attributes to a GC log record we
// could consider timing and reporting how long we wait in the
......@@ -1538,6 +1555,10 @@ bool G1CollectedHeap::do_collection(bool explicit_gc,
post_full_gc_dump(gc_timer);
if (G1ElasticHeap) {
elastic_heap()->record_gc_end();
}
gc_timer->register_gc_end();
gc_tracer->report_gc_end(gc_timer->gc_end(), gc_timer->time_partitions());
}
......@@ -1560,6 +1581,10 @@ void G1CollectedHeap::do_full_collection(bool clear_all_soft_refs) {
void
G1CollectedHeap::
resize_if_necessary_after_full_collection(size_t word_size) {
if (G1ElasticHeap) {
// We never resize heap in full GC with elastic heap
return;
}
// Include the current allocation, if any, and bytes that will be
// pre-allocated to support collections, as "used".
const size_t used_after_gc = used();
......@@ -1762,6 +1787,11 @@ bool G1CollectedHeap::expand(size_t expand_bytes) {
return false;
}
if (G1ElasticHeap && G1CollectedHeap::heap()->elastic_heap() != NULL) {
// Heap initialization completes. Don't expand ever.
return false;
}
uint regions_to_expand = (uint)(aligned_expand_bytes / HeapRegion::GrainBytes);
assert(regions_to_expand > 0, "Must expand by at least one region");
......@@ -1860,6 +1890,7 @@ G1CollectedHeap::G1CollectedHeap(G1CollectorPolicy* policy_) :
_has_humongous_reclaim_candidates(false),
_free_regions_coming(false),
_young_list(new YoungList(this)),
_elastic_heap(NULL),
_gc_time_stamp(0),
_survivor_plab_stats(YoungPLABSize, PLABWeight),
_old_plab_stats(OldPLABSize, PLABWeight),
......@@ -1952,6 +1983,19 @@ jint G1CollectedHeap::initialize() {
size_t max_byte_size = collector_policy()->max_heap_byte_size();
size_t heap_alignment = collector_policy()->heap_alignment();
if (G1ElasticHeap) {
size_t page_size = UseLargePages ? os::large_page_size() : os::vm_page_size();
if (HeapRegion::GrainBytes < page_size || (HeapRegion::GrainBytes % page_size) != 0) {
vm_exit_during_initialization(err_msg("G1ElasticHeap requires G1HeapRegionSize("
SIZE_FORMAT
" bytes) is multiple times of OS page size("
SIZE_FORMAT
" bytes)",
HeapRegion::GrainBytes, page_size));
return JNI_EINVAL;
}
}
// Ensure that the sizes are properly aligned.
Universe::check_alignment(init_byte_size, HeapRegion::GrainBytes, "g1 heap");
Universe::check_alignment(max_byte_size, HeapRegion::GrainBytes, "g1 heap");
......@@ -2081,6 +2125,10 @@ jint G1CollectedHeap::initialize() {
// Perform any initialization actions delegated to the policy.
g1_policy()->init();
if (G1ElasticHeap) {
_elastic_heap = new ElasticHeap(this);
}
JavaThread::satb_mark_queue_set().initialize(SATB_Q_CBL_mon,
SATB_Q_FL_lock,
G1SATBProcessCompletedThreshold,
......@@ -2145,6 +2193,9 @@ void G1CollectedHeap::stop() {
// that are destroyed during shutdown.
_cg1r->stop();
_cmThread->stop();
if (G1ElasticHeap) {
elastic_heap()->destroy();
}
if (G1StringDedup::is_enabled()) {
G1StringDedup::stop();
}
......@@ -2522,6 +2573,7 @@ void G1CollectedHeap::collect(GCCause::Cause cause) {
}
} else {
if (cause == GCCause::_gc_locker || cause == GCCause::_wb_young_gc
|| cause == GCCause::_g1_elastic_heap_trigger_gc
DEBUG_ONLY(|| cause == GCCause::_scavenge_alot)) {
// Schedule a standard evacuation pause. We're setting word_size
......@@ -4032,6 +4084,10 @@ G1CollectedHeap::do_collection_pause_at_safepoint(double target_pause_time_ms) {
{ // Call to jvmpi::post_class_unload_events must occur outside of active GC
IsGCActiveMark x;
if (G1ElasticHeap) {
elastic_heap()->prepare_in_gc_start();
}
gc_prologue(false);
increment_total_collections(false /* full gc */);
increment_gc_time_stamp();
......@@ -4243,6 +4299,10 @@ G1CollectedHeap::do_collection_pause_at_safepoint(double target_pause_time_ms) {
_cm->verify_no_cset_oops();
_cm->note_end_of_gc();
if (G1ElasticHeap) {
elastic_heap()->perform();
}
// This timing is only used by the ergonomics to handle our pause target.
// It is unclear why this should not include the full pause. We will
// investigate this in CR 7178365.
......
......@@ -44,6 +44,7 @@
#include "memory/memRegion.hpp"
#include "memory/sharedHeap.hpp"
#include "utilities/stack.hpp"
#include "gc_implementation/g1/elasticHeap.hpp"
// A "G1CollectedHeap" is an implementation of a java heap for HotSpot.
// It uses the "Garbage First" heap organization and algorithm, which
......@@ -75,6 +76,7 @@ class G1NewTracer;
class G1OldTracer;
class EvacuationFailedInfo;
class nmethod;
class ElasticHeap;
typedef OverflowTaskQueue<StarTask, mtGC> RefToScanQueue;
typedef GenericTaskQueueSet<RefToScanQueue, mtGC> RefToScanQueueSet;
......@@ -184,6 +186,8 @@ class G1CollectedHeap : public SharedHeap {
friend class MutatorAllocRegion;
friend class SurvivorGCAllocRegion;
friend class OldGCAllocRegion;
friend class ElasticHeap;
friend class ElasticHeapEvaluator;
friend class G1Allocator;
friend class G1DefaultAllocator;
friend class G1ResManAllocator;
......@@ -447,6 +451,8 @@ protected:
// The young region list.
YoungList* _young_list;
ElasticHeap* _elastic_heap;
// The current policy object for the collector.
G1CollectorPolicy* _g1_policy;
......@@ -1101,7 +1107,13 @@ public:
}
// The current number of regions in the heap.
uint num_regions() const { return _hrm.length(); }
uint num_regions() const {
if (G1ElasticHeap && elastic_heap() != NULL && !elastic_heap()->heap_capacity_changed()) {
return _hrm.max_length();
} else {
return _hrm.length();
}
}
// The max number of regions in the heap.
uint max_regions() const { return _hrm.max_length(); }
......@@ -1482,6 +1494,8 @@ public:
YoungList* young_list() const { return _young_list; }
ElasticHeap* elastic_heap() const { return _elastic_heap; }
// debugging
bool check_young_list_well_formed() {
return _young_list->check_list_well_formed();
......
......@@ -41,6 +41,7 @@
#include "runtime/java.hpp"
#include "runtime/mutexLocker.hpp"
#include "utilities/debug.hpp"
#include "gc_implementation/g1/elasticHeap.hpp"
// Different defaults for different number of GC threads
// They were chosen by running GCOld and SPECjbb on debris with different
......@@ -1483,6 +1484,12 @@ void G1CollectorPolicy::update_survivors_policy() {
// smaller than 1.0) we'll get 1.
_max_survivor_regions = (uint) ceil(max_survivor_regions_d);
if (G1ElasticHeap && _g1->elastic_heap()->is_gc_to_resize_young_gen()) {
// If elastic heap will shrink the young gen
// we need to make the survivor less than the target young gen size
_max_survivor_regions = MIN2(_max_survivor_regions, _g1->elastic_heap()->max_available_survivor_regions());
}
_tenuring_threshold = _survivors_age_table.compute_tenuring_threshold(
HeapRegion::GrainWords * _max_survivor_regions);
}
......
......@@ -39,6 +39,7 @@
class HeapRegion;
class CollectionSetChooser;
class G1GCPhaseTimes;
class ElasticHeap;
// TraceGen0Time collects data on _both_ young and mixed evacuation pauses
// (the latter may contain non-young regions - i.e. regions that are
......@@ -158,9 +159,17 @@ public:
bool adaptive_young_list_length() {
return _adaptive_size;
}
void resize_min_desired_young_length(uint size) {
_min_desired_young_length = size;
}
void resize_max_desired_young_length(uint size) {
_max_desired_young_length = size;
}
};
class G1CollectorPolicy: public CollectorPolicy {
friend class ElasticHeap;
friend class ElasticHeapEvaluator;
private:
static G1IHOPControl* create_ihop_control();
......@@ -843,6 +852,10 @@ public:
_gcs_are_young = gcs_are_young;
}
bool last_young_gc() const {
return _last_young_gc;
}
bool adaptive_young_list_length() {
return _young_gen_sizer->adaptive_young_list_length();
}
......
......@@ -224,6 +224,21 @@ bool G1PageBasedVirtualSpace::commit(size_t start_page, size_t size_in_pages) {
return zero_filled;
}
void G1PageBasedVirtualSpace::par_commit(size_t start_page, size_t size_in_pages, bool allow_pretouch) {
// We need to make sure to commit all pages covered by the given area.
guarantee(is_area_uncommitted(start_page, size_in_pages), "Specified area is not uncommitted");
guarantee(!_special, "sanity");
size_t end_page = start_page + size_in_pages;
commit_internal(start_page, end_page);
_committed.par_set_range(start_page, end_page, BitMap::unknown_range);
if (AlwaysPreTouch && allow_pretouch) {
pretouch_internal(start_page, end_page);
}
}
void G1PageBasedVirtualSpace::uncommit_internal(size_t start_page, size_t end_page) {
guarantee(start_page < end_page,
err_msg("Given start page " SIZE_FORMAT " is larger or equal to end page " SIZE_FORMAT, start_page, end_page));
......@@ -232,6 +247,14 @@ void G1PageBasedVirtualSpace::uncommit_internal(size_t start_page, size_t end_pa
os::uncommit_memory(start_addr, pointer_delta(bounded_end_addr(end_page), start_addr, sizeof(char)));
}
void G1PageBasedVirtualSpace::free_memory_internal(size_t start_page, size_t end_page) {
guarantee(start_page < end_page,
err_msg("Given start page " SIZE_FORMAT " is larger or equal to end page " SIZE_FORMAT, start_page, end_page));
char* start_addr = page_start(start_page);
os::free_memory(start_addr, pointer_delta(bounded_end_addr(end_page), start_addr, sizeof(char)), _page_size);
}
void G1PageBasedVirtualSpace::uncommit(size_t start_page, size_t size_in_pages) {
guarantee(is_area_committed(start_page, size_in_pages), "checking");
......@@ -247,6 +270,23 @@ void G1PageBasedVirtualSpace::uncommit(size_t start_page, size_t size_in_pages)
_committed.clear_range(start_page, end_page);
}
void G1PageBasedVirtualSpace::par_uncommit(size_t start_page, size_t size_in_pages) {
guarantee(is_area_committed(start_page, size_in_pages), "checking");
guarantee(!_special, "sanity");
size_t end_page = start_page + size_in_pages;
uncommit_internal(start_page, end_page);
_committed.par_clear_range(start_page, end_page, BitMap::unknown_range);
}
void G1PageBasedVirtualSpace::free_memory(size_t start_page, size_t size_in_pages) {
guarantee(is_area_committed(start_page, size_in_pages), "checking");
guarantee(!_special, "sanity");
size_t end_page = start_page + size_in_pages;
free_memory_internal(start_page, end_page);
}
bool G1PageBasedVirtualSpace::contains(const void* p) const {
return _low_boundary <= (const char*) p && (const char*) p < _high_boundary;
}
......
......@@ -85,6 +85,9 @@ class G1PageBasedVirtualSpace VALUE_OBJ_CLASS_SPEC {
// Uncommit the given memory range.
void uncommit_internal(size_t start_page, size_t end_page);
// Free the given memory range.
void free_memory_internal(size_t start_page, size_t end_page);
// Pretouch the given memory range.
void pretouch_internal(size_t start_page, size_t end_page);
......@@ -117,6 +120,15 @@ class G1PageBasedVirtualSpace VALUE_OBJ_CLASS_SPEC {
// Uncommit the given area of pages starting at start being size_in_pages large.
void uncommit(size_t start_page, size_t size_in_pages);
// MT-safe commit the given area of pages starting at start being size_in_pages large.
void par_commit(size_t start_page, size_t size_in_pages, bool allow_pretouch = true);
// MT-safe uncommit the given area of pages starting at start being size_in_pages large.
void par_uncommit(size_t start_page, size_t size_in_pages);
// Free the given area of pages starting at start being size_in_pages large.
void free_memory(size_t start_page, size_t size_in_pages);
// Initialize the given reserved space with the given base address and the size
// actually used.
// Prefer to commit in page_size chunks.
......
......@@ -76,6 +76,21 @@ class G1RegionsLargerThanCommitSizeMapper : public G1RegionToSpaceMapper {
_storage.uncommit((size_t)start_idx * _pages_per_region, num_regions * _pages_per_region);
_commit_map.clear_range(start_idx, start_idx + num_regions);
}
virtual void par_commit_region_memory(uint idx) {
_storage.par_commit((size_t)idx * _pages_per_region, _pages_per_region, false);
_commit_map.par_set_range(idx, idx + 1, BitMap::unknown_range);
}
virtual void par_uncommit_region_memory(uint idx) {
_storage.par_uncommit((size_t)idx * _pages_per_region, _pages_per_region);
_commit_map.par_clear_range(idx, idx + 1, BitMap::unknown_range);
}
virtual void free_region_memory(uint idx) {
_storage.free_memory((size_t)idx * _pages_per_region, _pages_per_region);
}
};
// G1RegionToSpaceMapper implementation where the region granularity is smaller
......@@ -139,6 +154,18 @@ class G1RegionsSmallerThanCommitSizeMapper : public G1RegionToSpaceMapper {
_commit_map.clear_bit(i);
}
}
virtual void par_commit_region_memory(uint idx) {
ShouldNotReachHere();
}
virtual void par_uncommit_region_memory(uint idx) {
ShouldNotReachHere();
}
virtual void free_region_memory(uint idx) {
ShouldNotReachHere();
}
};
void G1RegionToSpaceMapper::fire_on_commit(uint start_idx, size_t num_regions, bool zero_filled) {
......
......@@ -72,6 +72,9 @@ class G1RegionToSpaceMapper : public CHeapObj<mtGC> {
virtual void commit_regions(uint start_idx, size_t num_regions = 1) = 0;
virtual void uncommit_regions(uint start_idx, size_t num_regions = 1) = 0;
virtual void par_commit_region_memory(uint idx) = 0;
virtual void par_uncommit_region_memory(uint idx) = 0;
virtual void free_region_memory(uint idx) = 0;
// Creates an appropriate G1RegionToSpaceMapper for the given parameters.
// The actual space to be used within the given reservation is given by actual_size.
......
......@@ -29,6 +29,8 @@
#include "gc_implementation/g1/g1CollectedHeap.inline.hpp"
#include "gc_implementation/g1/concurrentG1Refine.hpp"
#include "memory/allocation.hpp"
#include "runtime/os.hpp"
#include "gc_implementation/g1/elasticHeap.hpp"
void HeapRegionManager::initialize(G1RegionToSpaceMapper* heap_storage,
G1RegionToSpaceMapper* prev_bitmap,
......@@ -91,6 +93,33 @@ void HeapRegionManager::commit_regions(uint index, size_t num_regions) {
_card_counts_mapper->commit_regions(index, num_regions);
}
void HeapRegionManager::uncommit_region_memory(uint idx) {
assert(G1ElasticHeap, "Precondition");
// Print before uncommitting.
if (G1CollectedHeap::heap()->hr_printer()->is_active()) {
HeapRegion* hr = at(idx);
G1CollectedHeap::heap()->hr_printer()->uncommit(hr->bottom(), hr->end());
}
_heap_mapper->par_uncommit_region_memory(idx);
}
void HeapRegionManager::commit_region_memory(uint idx) {
assert(G1ElasticHeap, "Precondition");
// Print before committing.
if (G1CollectedHeap::heap()->hr_printer()->is_active()) {
HeapRegion* hr = at(idx);
G1CollectedHeap::heap()->hr_printer()->commit(hr->bottom(), hr->end());
}
_heap_mapper->par_commit_region_memory(idx);
}
void HeapRegionManager::free_region_memory(uint idx) {
assert(G1ElasticHeap, "Precondition");
_heap_mapper->free_region_memory(idx);
}
void HeapRegionManager::uncommit_regions(uint start, size_t num_regions) {
guarantee(num_regions >= 1, err_msg("Need to specify at least one region to uncommit, tried to uncommit zero regions at %u", start));
guarantee(_num_committed >= num_regions, "pre-condition");
......@@ -172,6 +201,11 @@ uint HeapRegionManager::expand_at(uint start, uint num_regions) {
return 0;
}
if (G1ElasticHeap && G1CollectedHeap::heap()->elastic_heap() != NULL) {
// Heap initialization completes. Don't expand ever.
return 0;
}
uint cur = start;
uint idx_last_found = 0;
uint num_last_found = 0;
......@@ -197,7 +231,7 @@ uint HeapRegionManager::find_contiguous(size_t num, bool empty_only) {
while (length_found < num && cur < max_length()) {
HeapRegion* hr = _regions.get_by_index(cur);
if ((!empty_only && !is_available(cur)) || (is_available(cur) && hr != NULL && hr->is_empty())) {
if ((!empty_only && !is_available(cur) && !G1ElasticHeap) || (is_available(cur) && hr != NULL && hr->is_empty())) {
// This region is a potential candidate for allocation into.
length_found++;
} else {
......@@ -353,6 +387,135 @@ void HeapRegionManager::par_iterate(HeapRegionClosure* blk, uint worker_id, uint
}
}
void HeapRegionManager::commit_region_memory(FreeRegionList* list) {
assert(G1ElasticHeap, "Precondition");
FreeRegionListIterator iter(list);
while (iter.more_available()) {
HeapRegion* hr = iter.get_next();
commit_region_memory(hr->hrm_index());
}
}
void HeapRegionManager::uncommit_region_memory(FreeRegionList* list) {
assert(G1ElasticHeap, "Precondition");
FreeRegionListIterator iter(list);
while (iter.more_available()) {
HeapRegion* hr = iter.get_next();
uncommit_region_memory(hr->hrm_index());
}
}
void HeapRegionManager::set_region_available(FreeRegionList* list) {
assert(G1ElasticHeap, "Precondition");
FreeRegionListIterator iter(list);
while (iter.more_available()) {
HeapRegion* hr = iter.get_next();
_available_map.par_set_range(hr->hrm_index(), hr->hrm_index() + 1, BitMap::unknown_range);
}
_num_committed += list->length();
}
void HeapRegionManager::set_region_unavailable(FreeRegionList* list) {
assert(G1ElasticHeap, "Precondition");
FreeRegionListIterator iter(list);
while (iter.more_available()) {
HeapRegion* hr = iter.get_next();
_available_map.par_clear_range(hr->hrm_index(), hr->hrm_index() + 1, BitMap::unknown_range);
}
assert(_num_committed > list->length(), "sanity");
_num_committed -= list->length();
}
uint HeapRegionManager::num_uncommitted_regions() {
assert(G1ElasticHeap, "Precondition");
return _uncommitted_list.length();
}
void HeapRegionManager::recover_uncommitted_regions() {
assert(G1ElasticHeap, "Precondition");
assert_at_safepoint(true /* should_be_vm_thread */);
commit_region_memory(&_uncommitted_list);
set_region_available(&_uncommitted_list);
_free_list.add_ordered(&_uncommitted_list);
}
void HeapRegionManager::move_to_uncommitted_list(FreeRegionList* list) {
assert(G1ElasticHeap, "Precondition");
assert_at_safepoint(true /* should_be_vm_thread */);
_uncommitted_list.add_ordered(list);
}
void HeapRegionManager::move_to_free_list(FreeRegionList* list) {
assert(G1ElasticHeap, "Precondition");
assert_at_safepoint(true /* should_be_vm_thread */);
set_region_available(list);
_free_list.add_ordered(list);
}
void HeapRegionManager::prepare_uncommit_regions(FreeRegionList* list, uint num) {
assert(G1ElasticHeap, "Precondition");
assert(num <= _free_list.length(), "sanity");
assert_at_safepoint(true /* should_be_vm_thread */);
assert(list->is_empty(), "sanity");
G1CollectedHeap* g1h = G1CollectedHeap::heap();
for (uint i = 0; i < num; i++) {
HeapRegion* hr = _free_list.remove_region(false /* from_head */);
list->add_ordered(hr);
}
set_region_unavailable(list);
}
void HeapRegionManager::prepare_commit_regions(FreeRegionList* list, uint num) {
assert(G1ElasticHeap, "Precondition");
assert(num <= _uncommitted_list.length(), "sanity");
assert_at_safepoint(true /* should_be_vm_thread */);
assert(list->is_empty(), "sanity");
G1CollectedHeap* g1h = G1CollectedHeap::heap();
for (uint i = 0; i < num; i++) {
HeapRegion* hr = _uncommitted_list.remove_region(true /* from_head */);
assert(!is_available(hr->hrm_index()), "sanity");
list->add_ordered(hr);
}
}
void HeapRegionManager::prepare_old_region_list_to_free(FreeRegionList* to_free_list,
uint reserve_regions,
uint free_regions_for_young_gen) {
assert(G1ElasticHeap, "Precondition");
assert_at_safepoint(true /* should_be_vm_thread */);
assert(to_free_list->is_empty(), "sanity");
G1CollectedHeap* g1h = G1CollectedHeap::heap();
assert(!g1h->elastic_heap()->in_conc_cycle(), "Precondition");
if (_free_list.length() <= (reserve_regions + free_regions_for_young_gen)) {
// Not enough free regions
return;
}
uint uncommit_regions = _free_list.length() - reserve_regions - free_regions_for_young_gen;
FreeRegionListIterator iter(&_free_list);
// Skip reserve regions from the head of _free_list
for (uint i = 0; i < reserve_regions; i++) {
assert(iter.more_available(), "sanity");
iter.get_next();
}
// Remove regions out of free list for uncommit
for (uint i = 0; i < uncommit_regions; i++) {
assert(iter.more_available(), "sanity");
HeapRegion* hr = iter.remove_next();
to_free_list->add_ordered(hr);
}
set_region_unavailable(to_free_list);
}
uint HeapRegionManager::shrink_by(uint num_regions_to_remove) {
assert(length() > 0, "the region sequence should not be empty");
assert(length() <= _allocated_heapregions_length, "invariant");
......
......@@ -79,6 +79,7 @@ class HeapRegionManager: public CHeapObj<mtGC> {
FreeRegionList _free_list;
FreeRegionList _uncommitted_list;
// Each bit in this bitmap indicates that the corresponding region is available
// for allocation.
BitMap _available_map;
......@@ -131,7 +132,8 @@ public:
HeapRegionManager() : _regions(), _heap_mapper(NULL), _num_committed(0),
_next_bitmap_mapper(NULL), _prev_bitmap_mapper(NULL), _bot_mapper(NULL),
_allocated_heapregions_length(0), _available_map(),
_free_list("Free list", new MasterFreeRegionListMtSafeChecker())
_free_list("Free list", new MasterFreeRegionListMtSafeChecker()),
_uncommitted_list("Free list of uncommitted regions", new MasterFreeRegionListMtSafeChecker())
{ }
void initialize(G1RegionToSpaceMapper* heap_storage,
......@@ -232,6 +234,34 @@ public:
// Return the actual number of uncommitted regions.
uint shrink_by(uint num_regions_to_remove);
// Number of regions in uncommitted free list
uint num_uncommitted_regions();
// Move uncommitted regions back into free list
void recover_uncommitted_regions();
// Move regions into uncommitted list
void move_to_uncommitted_list(FreeRegionList* list);
// Move regions back into free list
void move_to_free_list(FreeRegionList* list);
// Remove regions from free list for uncommitment
void prepare_uncommit_regions(FreeRegionList* list, uint num);
// Remove regions from uncommitted list for commitment
void prepare_commit_regions(FreeRegionList* list, uint num);
// Remove old regions to free
void prepare_old_region_list_to_free(FreeRegionList* to_free_list,
uint reserve_regions,
uint free_regions_for_young_gen);
void commit_region_memory(FreeRegionList* list);
void uncommit_region_memory(FreeRegionList* list);
void set_region_available(FreeRegionList* list);
void set_region_unavailable(FreeRegionList* list);
void commit_region_memory(uint idx);
void uncommit_region_memory(uint idx);
void free_region_memory(uint idx);
void verify();
// Do some sanity checking.
......
......@@ -214,6 +214,9 @@ private:
inline HeapRegion* remove_from_head_impl();
inline HeapRegion* remove_from_tail_impl();
// Only called in FreeRegionListIterator so make it private
HeapRegion* remove_region(HeapRegion* hr);
protected:
virtual void fill_in_ext_msg_extra(hrs_ext_msg* msg);
......@@ -287,6 +290,19 @@ public:
return hr;
}
HeapRegion* remove_next() {
assert(more_available(),
"remove_next() should be called when more regions are available");
HeapRegion* hr = _curr;
_list->verify_region(hr);
_curr = hr->next();
hr = _list->remove_region(hr);
return hr;
}
FreeRegionListIterator(FreeRegionList* list) : _curr(NULL), _list(list) {
_curr = list->_head;
}
......
......@@ -148,5 +148,40 @@ inline HeapRegion* FreeRegionList::remove_region(bool from_head) {
return hr;
}
inline HeapRegion* FreeRegionList::remove_region(HeapRegion* hr) {
check_mt_safety();
verify_optional();
HeapRegion* prev = hr->prev();
HeapRegion* next = hr->next();
if (prev) {
prev->set_next(next);
} else {
// hr is _head
assert(_head == hr, "Sanity");
_head = next;
}
if (next) {
next->set_prev(prev);
} else {
// hr is _tail
assert(_tail == hr, "Sanity");
_tail = prev;
}
hr->set_prev(NULL);
hr->set_next(NULL);
if (_last == hr) {
_last = NULL;
}
// remove() will verify the region and check mt safety.
remove(hr);
return hr;
}
#endif // SHARE_VM_GC_IMPLEMENTATION_G1_HEAPREGIONSET_INLINE_HPP
......@@ -100,6 +100,9 @@ const char* GCCause::to_string(GCCause::Cause cause) {
case _g1_humongous_allocation:
return "G1 Humongous Allocation";
case _g1_elastic_heap_trigger_gc:
return "Elastic Heap triggered GC";
case _last_ditch_collection:
return "Last ditch collection";
......
......@@ -73,6 +73,8 @@ class GCCause : public AllStatic {
_g1_inc_collection_pause,
_g1_humongous_allocation,
_g1_elastic_heap_trigger_gc,
_last_ditch_collection,
_last_gc_cause
};
......
......@@ -78,6 +78,8 @@
#include "utilities/histogram.hpp"
#include "utilities/top.hpp"
#include "utilities/utf8.hpp"
#include "gc_implementation/g1/g1CollectedHeap.hpp"
#include "gc_implementation/g1/elasticHeap.hpp"
#ifdef TARGET_OS_FAMILY_linux
# include "jvm_linux.h"
#endif
......@@ -4748,3 +4750,89 @@ JVM_ENTRY(void, JVM_NotifyJVMDeoptWarmUpMethods(JNIEnv *env, jclass clazz))
}
}
JVM_END
JVM_ENTRY(jint, JVM_ElasticHeapGetEvaluationMode(JNIEnv *env, jclass klass))
JVMWrapper("JVM_ElasticHeapGetEvaluationMode");
assert(G1ElasticHeap, "Precondition");
return G1CollectedHeap::heap()->elastic_heap()->evaluation_mode();
JVM_END
JVM_ENTRY(void, JVM_ElasticHeapSetYoungGenCommitPercent(JNIEnv *env, jclass klass, jint percent))
JVMWrapper("JVM_ElasticHeapSetYoungGenCommitPercent");
assert(G1ElasticHeap, "Precondition");
ElasticHeap* elas = G1CollectedHeap::heap()->elastic_heap();
ElasticHeap::ErrorType error = elas->configure_setting(percent, ElasticHeap::ignore_arg(),
ElasticHeap::ignore_arg());
if (error == ElasticHeap::IllegalYoungPercent) {
char as_chars[256];
jio_snprintf(as_chars, sizeof(as_chars), "percent should be 0, or between %u and 100 ", ElasticHeapMinYoungCommitPercent);
THROW_MSG(vmSymbols::java_lang_IllegalArgumentException(), as_chars);
} else if (error != ElasticHeap::NoError) {
THROW_MSG(vmSymbols::java_lang_IllegalStateException(), ElasticHeap::to_string(error));
}
JVM_END
JVM_ENTRY(jint, JVM_ElasticHeapGetYoungGenCommitPercent(JNIEnv *env, jclass klass))
JVMWrapper("JVM_ElasticHeapGetYoungGenCommitPercent");
assert(G1ElasticHeap, "Precondition");
return G1CollectedHeap::heap()->elastic_heap()->young_commit_percent();
JVM_END
JVM_ENTRY(void, JVM_ElasticHeapSetUncommitIHOP(JNIEnv *env, jclass klass, jint percent))
JVMWrapper("JVM_ElasticHeapSetUncommitIHOP");
assert(G1ElasticHeap, "Precondition");
assert(percent >= 0 && percent <= 100, "sanity");
ElasticHeap* elas = G1CollectedHeap::heap()->elastic_heap();
ElasticHeap::ErrorType error = elas->configure_setting(ElasticHeap::ignore_arg(), percent,
ElasticHeap::ignore_arg());
if (error != ElasticHeap::NoError) {
THROW_MSG(vmSymbols::java_lang_IllegalStateException(), ElasticHeap::to_string(error));
}
JVM_END
JVM_ENTRY(jint, JVM_ElasticHeapGetUncommitIHOP(JNIEnv *env, jclass klass))
JVMWrapper("JVM_ElasticHeapGetUncommitIHOP");
assert(G1ElasticHeap, "Precondition");
return G1CollectedHeap::heap()->elastic_heap()->uncommit_ihop();
JVM_END
JVM_ENTRY(jlong, JVM_ElasticHeapGetTotalYoungUncommittedBytes(JNIEnv *env, jclass klass))
JVMWrapper("JVM_ElasticHeapGetTotalYoungUncommittedBytes");
assert(G1ElasticHeap, "Precondition");
return G1CollectedHeap::heap()->elastic_heap()->young_uncommitted_bytes();
JVM_END
JVM_ENTRY(void, JVM_ElasticHeapSetSoftmxPercent(JNIEnv *env, jclass klass, jint percent))
JVMWrapper("JVM_ElasticHeapSetSoftmxPercent");
assert(G1ElasticHeap, "Precondition");
assert(percent >= 0 && percent <= 100, "sanity");
ElasticHeap* elas = G1CollectedHeap::heap()->elastic_heap();
ElasticHeap::ErrorType error = elas->configure_setting(ElasticHeap::ignore_arg(),
ElasticHeap::ignore_arg(), percent);
if (error != ElasticHeap::NoError) {
THROW_MSG(vmSymbols::java_lang_IllegalStateException(), ElasticHeap::to_string(error));
}
JVM_END
JVM_ENTRY(jint, JVM_ElasticHeapGetSoftmxPercent(JNIEnv *env, jclass klass))
JVMWrapper("JVM_ElasticHeapGetSoftmxPercent");
assert(G1ElasticHeap, "Precondition");
return G1CollectedHeap::heap()->elastic_heap()->softmx_percent();
JVM_END
JVM_ENTRY(jlong, JVM_ElasticHeapGetTotalUncommittedBytes(JNIEnv *env, jclass klass))
JVMWrapper("JVM_ElasticHeapGetTotalUncommittedBytes");
assert(G1ElasticHeap, "Precondition");
return G1CollectedHeap::heap()->elastic_heap()->uncommitted_bytes();
JVM_END
......@@ -1714,6 +1714,28 @@ JVM_CheckJWarmUpCompilationIsComplete(JNIEnv* env, jclass ignored);
JNIEXPORT void JNICALL
JVM_NotifyJVMDeoptWarmUpMethods(JNIEnv* env, jclass clz);
/*
* com.alibaba.management.ElasticHeapMXBeanImpl
*/
JNIEXPORT jint JNICALL
JVM_ElasticHeapGetEvaluationMode(JNIEnv *env, jclass clazz);
JNIEXPORT void JNICALL
JVM_ElasticHeapSetYoungGenCommitPercent(JNIEnv *env, jclass klass, jint percent);
JNIEXPORT jint JNICALL
JVM_ElasticHeapGetYoungGenCommitPercent(JNIEnv *env, jclass klass);
JNIEXPORT void JNICALL
JVM_ElasticHeapSetUncommitIHOP(JNIEnv *env, jclass clazz, jint percent);
JNIEXPORT jint JNICALL
JVM_ElasticHeapGetUncommitIHOP(JNIEnv *env, jclass clazz);
JNIEXPORT jlong JNICALL
JVM_ElasticHeapGetTotalYoungUncommittedBytes(JNIEnv *env, jclass klass);
JNIEXPORT void JNICALL
JVM_ElasticHeapSetSoftmxPercent(JNIEnv *env, jclass clazz, jint percent);
JNIEXPORT jint JNICALL
JVM_ElasticHeapGetSoftmxPercent(JNIEnv *env, jclass clazz);
JNIEXPORT jlong JNICALL
JVM_ElasticHeapGetTotalUncommittedBytes(JNIEnv *env, jclass clazz);
#ifdef __cplusplus
} /* extern "C" */
#endif /* __cplusplus */
......
......@@ -2295,6 +2295,33 @@ bool Arguments::check_vm_args_consistency() {
// Note: Needs platform-dependent factoring.
bool status = true;
if (G1ElasticHeap) {
if (!UseG1GC) {
vm_exit_during_initialization("G1ElasticHeap only works with UseG1GC");
}
status = status && verify_interval(ElasticHeapMinYoungCommitPercent, 1, 100, "ElasticHeapMinYoungCommitPercent");
status = status && verify_interval(ElasticHeapPeriodicMinYoungCommitPercent, ElasticHeapMinYoungCommitPercent, 100, "ElasticHeapMinYoungCommitPercentAuto");
PropertyList_unique_add(&_system_properties, "com.alibaba.jvm.gc.ElasticHeapEnabled", (char*)"true");
status = status && verify_interval(ElasticHeapOldGenReservePercent, 1, 100, "ElasticHeapOldGenReservePercent");
if (InitialHeapSize != MaxHeapSize) {
jio_fprintf(defaultStream::error_stream(),
"G1ElasticHeap requires Xms:"
SIZE_FORMAT
" bytes same to Xmx: "
SIZE_FORMAT
" bytes",
InitialHeapSize, MaxHeapSize);
status = false;
}
}
if (ElasticHeapPeriodicUncommit) {
if (!G1ElasticHeap) {
vm_exit_during_initialization("ElasticHeapPeriodicUncommit only works with G1ElasticHeap");
}
}
// Allow both -XX:-UseStackBanging and -XX:-UseBoundThreads in non-product
// builds so the cost of stack banging can be measured.
#if (defined(PRODUCT) && defined(SOLARIS))
......
......@@ -27,17 +27,73 @@
// globals_extension.hpp extension
#define AJVM_FLAGS(develop, develop_pd, product, product_pd, diagnostic, experimental, notproduct, manageable, product_rw, lp64_product) \
\
manageable(bool, PrintYoungGenHistoAfterParNewGC, false, \
"print the young generation class histogram after parNew GC") \
\
manageable(bool, PrintGCRootsTraceTime, false, \
"Print GC Trace Time") \
\
manageable(intx, ArrayAllocationWarningSize, 512*M, \
"Desired size of array space allocation before " \
"printing a warning") \
\
\
manageable(bool, PrintYoungGenHistoAfterParNewGC, false, \
"print the young generation class histogram after parNew GC") \
\
manageable(bool, PrintGCRootsTraceTime, false, \
"Print GC Trace Time") \
\
manageable(intx, ArrayAllocationWarningSize, 512*M, \
"Desired size of array space allocation before " \
"printing a warning") \
\
product(bool, G1ElasticHeap, false, \
"Allow java heap to be resized in runtime") \
\
manageable(uintx, ElasticHeapMinYoungCommitPercent, 10, \
"Minimal commit percentage of young gen size") \
/* Similar to G1NewSizePercent/G1MaxNewSizePercent */ \
\
manageable(uintx, ElasticHeapYGCIntervalMinMillis, 5000, \
"Might uncommit memory only if young GC interval " \
"larger than this threshold in milliseconds ") \
\
manageable(uintx, ElasticHeapInitialMarkIntervalMinMillis, 60000, \
"Might uncommit memory only if initial mark interval " \
"larger than this threshold in milliseconds ") \
\
manageable(bool, ElasticHeapPeriodicUncommit, false, \
"Uncommit memory by periodic GC") \
\
manageable(uintx, ElasticHeapPeriodicUncommitStartupDelay, 300, \
"Starup delay in seconds for periodic uncommit") \
\
manageable(uintx, ElasticHeapPeriodicMinYoungCommitPercent, 50, \
"Minimal commit percentage of young gen in periodic gc mode") \
\
manageable(uintx, ElasticHeapPeriodicYGCIntervalMillis, 15000, \
"Target young gc interval in milliseconds after " \
"resizing young gen in periodic gc mode") \
\
manageable(uintx, ElasticHeapPeriodicInitialMarkIntervalMillis, 3600000, \
"Target initial mark interval " \
"in milliseconds in periodic gc mode. " \
"Free regions after mixed gc will be uncommitted. ") \
\
manageable(uintx, ElasticHeapPeriodicYGCIntervalCeilingPercent, 25, \
"Ceiling percent of the young gc inverval larger than " \
"ElasticHeapPeriodicYGCIntervalMillis") \
\
manageable(uintx, ElasticHeapPeriodicYGCIntervalFloorPercent, 25, \
"Floor percent of the young gc interval less than " \
"ElasticHeapPeriodicYGCIntervalMillis") \
\
manageable(uintx, ElasticHeapEagerMixedGCIntervalMillis, 15000, \
"Mixed GC will be triggered if desired mixed gc doesn't happen " \
"after the interval in milliseconds") \
\
manageable(uintx, ElasticHeapOldGenReservePercent, 5, \
"Percentage(0-100) of heap size to be reserved for " \
"old gen to grow") \
\
manageable(bool, PrintElasticHeapDetails, true, \
"Print Elastic Heap detail information in GC log") \
\
product(uintx, ElasticHeapParallelWorkers, 0, \
"Number of parallel worker threads for memory " \
"commit/uncommit. 0 be same as ConcGCThreads") \
//add new AJVM specific flags here
......
......@@ -102,6 +102,7 @@
#include "gc_implementation/concurrentMarkSweep/concurrentMarkSweepThread.hpp"
#include "gc_implementation/g1/concurrentMarkThread.inline.hpp"
#include "gc_implementation/parallelScavenge/pcTasks.hpp"
#include "gc_implementation/g1/elasticHeap.hpp"
#endif // INCLUDE_ALL_GCS
#ifdef COMPILER1
#include "c1/c1_Compiler.hpp"
......@@ -3625,6 +3626,12 @@ jint Threads::create_vm(JavaVMInitArgs* args, bool* canTryAgain) {
vm_exit_during_initialization(Handle(THREAD, PENDING_EXCEPTION));
}
}
if (G1ElasticHeap) {
ElasticHeapTimer::start();
if (HAS_PENDING_EXCEPTION) {
vm_exit_during_initialization(Handle(THREAD, PENDING_EXCEPTION));
}
}
#endif // INCLUDE_ALL_GCS
// Always call even when there are not JVMTI environments yet, since environments
......
......@@ -36,6 +36,8 @@
#include "services/attachListener.hpp"
#include "services/diagnosticCommand.hpp"
#include "services/heapDumper.hpp"
#include "gc_implementation/g1/g1CollectedHeap.hpp"
#include "gc_implementation/g1/elasticHeap.hpp"
volatile bool AttachListener::_initialized;
......@@ -257,6 +259,13 @@ static jint set_bool_flag(const char* name, AttachOperation* op, outputStream* o
}
value = (tmp != 0);
}
if (strcmp(name, "ElasticHeapPeriodicUncommit") == 0 & value) {
if (G1ElasticHeap &&
!G1CollectedHeap::heap()->elastic_heap()->can_turn_on_periodic_uncommit()) {
out->print_cr("cannot be set because of illegal state.");
return JNI_ERR;
}
}
bool res = CommandLineFlags::boolAtPut((char*)name, &value, Flag::ATTACH_ON_DEMAND);
if (! res) {
out->print_cr("setting flag %s failed", name);
......@@ -312,6 +321,16 @@ static jint set_uintx_flag(const char* name, AttachOperation* op, outputStream*
return JNI_ERR;
}
}
if (strcmp(name, "ElasticHeapMinYoungCommitPercent") == 0 ||
strcmp(name, "ElasticHeapPeriodicMinYoungCommitPercent") == 0 ||
strcmp(name, "ElasticHeapPeriodicYGCIntervalCeilingPercent") == 0 ||
strcmp(name, "ElasticHeapPeriodicYGCIntervalFloorPercent") == 0) {
if (value < 1 || value > 100) {
out->print_cr("%s must be between 1 and 100", name);
return JNI_ERR;
}
}
bool res = CommandLineFlags::uintxAtPut((char*)name, &value, Flag::ATTACH_ON_DEMAND);
if (! res) {
out->print_cr("setting flag %s failed", name);
......
......@@ -34,6 +34,7 @@
#include "services/management.hpp"
#include "utilities/macros.hpp"
#include "oops/objArrayOop.hpp"
#include "gc_implementation/g1/elasticHeap.hpp"
PRAGMA_FORMAT_MUTE_WARNINGS_FOR_GCC
......@@ -73,6 +74,7 @@ void DCmdRegistrant::register_dcmds(){
DCmdFactory::register_DCmdFactory(new DCmdFactoryImpl<JMXStartLocalDCmd>(jmx_agent_export_flags, true,false));
DCmdFactory::register_DCmdFactory(new DCmdFactoryImpl<JMXStopRemoteDCmd>(jmx_agent_export_flags, true,false));
DCmdFactory::register_DCmdFactory(new DCmdFactoryImpl<ElasticHeapDCmd>(full_export, true, false));
}
#ifndef HAVE_EXTRA_DCMD
......@@ -848,4 +850,115 @@ void JWarmupDCmd::print_info() {
"-deopt: %s\n"
"-help: %s\n",
_notify_startup.description(), _check_compile_finished.description(), _deopt.description(), _help.description());
}
\ No newline at end of file
}
ElasticHeapDCmd::ElasticHeapDCmd(outputStream* output, bool heap) :
DCmdWithParser(output, heap),
_young_commit_percent("young_commit_percent",
"Percentage of committed size in young generation to be adjusted to",
"INT", false),
_uncommit_ihop("uncommit_ihop",
"Percentage of heap to trigger concurrent mark to uncommit memory",
"INT", false),
_softmx_percent("softmx_percent",
"Percentage of committed size of heap to be adjusted to",
"INT", false) {
_dcmdparser.add_dcmd_option(&_young_commit_percent);
_dcmdparser.add_dcmd_option(&_uncommit_ihop);
_dcmdparser.add_dcmd_option(&_softmx_percent);
}
int ElasticHeapDCmd::num_arguments() {
ResourceMark rm;
int num_args = 0;
ElasticHeapDCmd* dcmd = new ElasticHeapDCmd(NULL, false);
if (dcmd != NULL) {
DCmdMark mark(dcmd);
num_args = dcmd->_dcmdparser.num_arguments();
}
return num_args;
}
bool ElasticHeapDCmd::illegal_percent(uint percent, const char* name) {
if (percent > 100) {
output()->print_cr("Error: %s between 0 and 100.", name);
print_info();
return true;
}
return false;
}
void ElasticHeapDCmd::execute(DCmdSource source, TRAPS) {
if (!G1ElasticHeap) {
output()->print_cr("Error: -XX:+G1ElasticHeap is not enabled!");
return;
}
uint young_percent= ElasticHeap::ignore_arg();
uint uncommit_ihop = ElasticHeap::ignore_arg();
uint softmx_percent = ElasticHeap::ignore_arg();
bool option_set = false;
if (_young_commit_percent.is_set()) {
young_percent = _young_commit_percent.value();
option_set = true;
}
if (_uncommit_ihop.is_set()) {
uncommit_ihop = _uncommit_ihop.value();
if (illegal_percent(uncommit_ihop, "uncommit_ihop")) {
return;
}
option_set = true;
}
if (_softmx_percent.is_set()) {
if (_young_commit_percent.is_set() || _uncommit_ihop.is_set()) {
output()->print_cr("Error: softmx_percent should be set alone!");
print_info();
return;
}
softmx_percent = _softmx_percent.value();
if (illegal_percent(softmx_percent, "softmx_percent")) {
return;
}
option_set = true;
}
if (option_set) {
ElasticHeap::ErrorType error = G1CollectedHeap::heap()->elastic_heap()->configure_setting(young_percent, uncommit_ihop, softmx_percent);
if (error == ElasticHeap::IllegalMode) {
output()->print_cr("Error: not in correct mode.");
} else if (error == ElasticHeap::IllegalYoungPercent) {
output()->print_cr("Error: young_commit_percent should be 0, or between %d and 100", ElasticHeapMinYoungCommitPercent);
} else if (error != ElasticHeap::NoError) {
output()->print_cr("Error: command fails because %s", ElasticHeap::to_string(error));
} else {
// Success
}
}
print_info();
}
void ElasticHeapDCmd::print_info() {
uint percent;
jlong uncommitted_bytes;
ElasticHeap::EvaluationMode mode = G1CollectedHeap::heap()->elastic_heap()->evaluation_mode();
switch (mode) {
case ElasticHeap::InactiveMode:
output()->print_cr("[GC.elastic_heap: inactive]");
break;
case ElasticHeap::SoftmxMode:
output()->print_cr("[GC.elastic_heap: in %s mode]", ElasticHeap::to_string(mode));
percent = G1CollectedHeap::heap()->elastic_heap()->softmx_percent();
uncommitted_bytes = G1CollectedHeap::heap()->elastic_heap()->uncommitted_bytes();
output()->print_cr("[GC.elastic_heap: softmx percent %d, uncommitted memory %ld B]", percent, uncommitted_bytes);
break;
default:
output()->print_cr("[GC.elastic_heap: in %s mode]", ElasticHeap::to_string(mode));
percent = G1CollectedHeap::heap()->elastic_heap()->young_commit_percent();
uncommitted_bytes = G1CollectedHeap::heap()->elastic_heap()->young_uncommitted_bytes();
output()->print_cr("[GC.elastic_heap: young generation commit percent %d, uncommitted memory %ld B]", percent, uncommitted_bytes);
break;
}
}
......@@ -461,4 +461,23 @@ public:
virtual void execute(DCmdSource source, TRAPS);
};
class ElasticHeapDCmd : public DCmdWithParser {
protected:
DCmdArgument<jlong> _young_commit_percent;
DCmdArgument<jlong> _uncommit_ihop;
DCmdArgument<jlong> _softmx_percent;
void print_info();
bool illegal_percent(uint percent, const char* name);
public:
ElasticHeapDCmd(outputStream* output, bool heap_allocated);
static const char* name() {
return "GC.elastic_heap";
}
static const char* description() {
return "Elastic Heap Command";
}
static int num_arguments();
virtual void execute(DCmdSource source, TRAPS);
};
#endif // SHARE_VM_SERVICES_DIAGNOSTICCOMMAND_HPP
/*
* Copyright (c) 2019 Alibaba Group Holding Limited. All Rights Reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Alibaba designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*/
import java.io.BufferedReader;
import java.io.InputStreamReader;
import java.io.File;
import com.oracle.java.testlibrary.*;
/* @test
* @summary test elastic-heap conflict commands
* @library /testlibrary
* @build TestElasticHeapConflictCommand
* @run main/othervm/timeout=200
-XX:+UseG1GC -XX:+G1ElasticHeap -Xmx1000m -Xms1000m
-XX:ElasticHeapPeriodicYGCIntervalMillis=400
-XX:ElasticHeapPeriodicUncommitStartupDelay=0
-Xmn200m -XX:G1HeapRegionSize=1m -XX:+AlwaysPreTouch
-XX:ElasticHeapYGCIntervalMinMillis=50
-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps
TestElasticHeapConflictCommand
*/
public class TestElasticHeapConflictCommand {
public static void main(String[] args) throws Exception {
OutputAnalyzer output;
byte[] arr = new byte[200*1024];
// Allocate 200k per 1ms , 200M per second
// so 2 GCs per second
for (int i = 0; i < 1000 * 3; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
int rssFull = getRss();
System.out.println("rssFull: " + rssFull);
output = triggerJinfo("+ElasticHeapPeriodicUncommit");
output.shouldHaveExitValue(0);
// Allocate 200k per ms , 200M per second
// so 1 GC per second
for (int i = 0; i < 1000 * 5; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
output = triggerJcmd("GC.elastic_heap", "young_commit_percent=50");
System.out.println(output.getOutput());
output.shouldContain("Error: not in correct mode");
output.shouldHaveExitValue(0);
output = triggerJcmd("GC.elastic_heap", "uncommit_ihop=50");
System.out.println(output.getOutput());
output.shouldContain("Error: not in correct mode");
output.shouldHaveExitValue(0);
output = triggerJcmd("GC.elastic_heap", "softmx_percent=50");
System.out.println(output.getOutput());
output.shouldContain("not in correct mode");
output.shouldHaveExitValue(0);
output = triggerJinfo("-ElasticHeapPeriodicUncommit");
output.shouldHaveExitValue(0);
// Allocate 200k per ms , 200M per second
// so 1 GC per second
for (int i = 0; i < 1000 * 5; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
output = triggerJcmd("GC.elastic_heap", "young_commit_percent=50");
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: young generation commit percent 50, uncommitted memory 104857600 B]");
output.shouldHaveExitValue(0);
output = triggerJinfo("+ElasticHeapPeriodicUncommit");
System.out.println(output.getOutput());
output.shouldContain("cannot be set because of illegal state");
output.shouldHaveExitValue(1);
output = triggerJcmd("GC.elastic_heap", "softmx_percent=80");
System.out.println(output.getOutput());
output.shouldContain("Error: not in correct mode");
output.shouldHaveExitValue(0);
// Allocate 200k per ms , 200M per second
// so 1 GC per second
for (int i = 0; i < 1000 * 5; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
output = triggerJcmd("GC.elastic_heap", "young_commit_percent=0", "uncommit_ihop=30");
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: young generation commit percent 100, uncommitted memory 0 B]");
output.shouldHaveExitValue(0);
// Allocate 200k per ms , 200M per second
// so 1 GC per second
for (int i = 0; i < 1000 * 5; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
output = triggerJinfo("+ElasticHeapPeriodicUncommit");
System.out.println(output.getOutput());
output.shouldContain("cannot be set because of illegal state");
output.shouldHaveExitValue(1);
output = triggerJcmd("GC.elastic_heap", "softmx_percent=80");
System.out.println(output.getOutput());
output.shouldContain("Error: not in correct mode");
output.shouldHaveExitValue(0);
output = triggerJcmd("GC.elastic_heap", "uncommit_ihop=0");
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: inactive]");
output.shouldHaveExitValue(0);
// Allocate 200k per ms , 200M per second
// so 1 GC per second
for (int i = 0; i < 1000 * 5; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
output = triggerJcmd("GC.elastic_heap", "softmx_percent=80");
System.out.println(output.getOutput());
output.shouldContain("in softmx mode");
output.shouldHaveExitValue(0);
// Allocate 400k per ms , 400M per second
// so 2 GC per second
for (int i = 0; i < 1000 * 5; i++) {
arr = new byte[200*1024];
arr = new byte[200*1024];
Thread.sleep(1);
}
output = triggerJinfo("+ElasticHeapPeriodicUncommit");
System.out.println(output.getOutput());
output.shouldContain("cannot be set because of illegal state");
output.shouldHaveExitValue(1);
output = triggerJcmd("GC.elastic_heap", "young_commit_percent=80");
System.out.println(output.getOutput());
output.shouldContain("Error: not in correct mode");
output.shouldHaveExitValue(0);
output = triggerJcmd("GC.elastic_heap", "uncommit_ihop=30");
System.out.println(output.getOutput());
output.shouldContain("Error: not in correct mode");
output.shouldHaveExitValue(0);
}
private static OutputAnalyzer triggerJcmd(String arg1, String arg2) throws Exception {
return triggerJcmd(arg1, arg2, null);
}
private static OutputAnalyzer triggerJcmd(String arg1, String arg2, String arg3) throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
JDKToolLauncher jcmd = JDKToolLauncher.create("jcmd").addToolArg(pid);
if (arg1 != null) {
jcmd.addToolArg(arg1);
}
if (arg2 != null) {
jcmd.addToolArg(arg2);
}
if (arg3 != null) {
jcmd.addToolArg(arg3);
}
ProcessBuilder pb = new ProcessBuilder(jcmd.getCommand());
return new OutputAnalyzer(pb.start());
}
private static OutputAnalyzer triggerJinfo(String arg) throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
JDKToolLauncher jcmd = JDKToolLauncher.create("jinfo")
.addToolArg("-flag")
.addToolArg(arg)
.addToolArg(pid);
ProcessBuilder pb = new ProcessBuilder(jcmd.getCommand());
return new OutputAnalyzer(pb.start());
}
private static int getRss() throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
int rss = 0;
Process ps = Runtime.getRuntime().exec("cat /proc/"+pid+"/status");
ps.waitFor();
BufferedReader br = new BufferedReader(new InputStreamReader(ps.getInputStream()));
String line;
while (( line = br.readLine()) != null ) {
if (line.startsWith("VmRSS:") ) {
int numEnd = line.length() - 3;
int numBegin = line.lastIndexOf(" ", numEnd - 1) + 1;
rss = Integer.parseInt(line.substring(numBegin, numEnd));
break;
}
}
return rss;
}
}
/*
* Copyright (c) 2019 Alibaba Group Holding Limited. All Rights Reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Alibaba designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*/
import java.io.BufferedReader;
import java.io.InputStreamReader;
import java.io.File;
import com.oracle.java.testlibrary.*;
/* @test
* @summary test elastic-heap with jcmd
* @library /testlibrary
* @build TestElasticHeapGenerationLimit
* @run main/othervm/timeout=500
-XX:+UseG1GC -XX:+G1ElasticHeap -Xmx1000m -Xms1000m
-XX:MaxNewSize=400m -XX:G1HeapRegionSize=1m
-XX:ElasticHeapYGCIntervalMinMillis=500
-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps
TestElasticHeapGenerationLimit
*/
public class TestElasticHeapGenerationLimit {
public static void main(String[] args) throws Exception {
for (int i = 0; i < 3; i++) {
test();
}
}
public static void test() throws Exception {
byte[] arr = new byte[200*1024];
OutputAnalyzer output;
// Allocate 200k per 1ms, 200M per second
// so 0.5 GCs per second
for (int i = 0; i < 1000 * 5; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
int rssFull = getRss();
System.out.println("Full rss: " + rssFull);
System.gc();
output = triggerJcmd("GC.elastic_heap", "young_commit_percent=50");
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: young generation commit percent 50, uncommitted memory 209715200 B]");
output.shouldHaveExitValue(0);
for (int i = 0; i < 1000 * 5; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
int rss50 = getRss();
System.out.println("50% rss: " + rss50);
Asserts.assertTrue(rss50 < rssFull);
Asserts.assertTrue(Math.abs(rssFull - rss50) > 150 * 1024);
output = triggerJcmd("GC.elastic_heap", "young_commit_percent=100");
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: young generation commit percent 100, uncommitted memory 0 B]");
output.shouldHaveExitValue(0);
for (int i = 0; i < 1000 * 5; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
int rss100 = getRss();
System.out.println("100% rss: " + rss100);
Asserts.assertTrue(Math.abs(rss100 - rssFull) < 50 * 1024);
Asserts.assertTrue(Math.abs(rss100 - rss50) > 150 * 1024);
output = triggerJcmd("GC.elastic_heap", "young_commit_percent=0");
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: inactive]");
for (int i = 0; i < 1000 * 5; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
int rss0 = getRss();
System.out.println("Recover rss: " + rss0);
Asserts.assertTrue(Math.abs(rss0 - rssFull) < 50 * 1024);
Asserts.assertTrue(Math.abs(rss0 - rss50) > 150 * 1024);
output = triggerJcmd("GC.elastic_heap", "young_commit_percent=30");
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: young generation commit percent 30, uncommitted memory 293601280 B]");
for (int i = 0; i < 1000 * 5; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
System.gc();
int rss30 = getRss();
System.out.println("rss30: " + rss30);
Asserts.assertTrue(rss30 < rss50);
Asserts.assertTrue(Math.abs(rss30 - rssFull) > 250 * 1024);
for (int i = 0; i < 1000 * 5; i++) {
arr = new byte[200*1024];
arr = new byte[200*1024];
Thread.sleep(1);
}
output = triggerJcmd("GC.elastic_heap", "young_commit_percent=20");
System.out.println(output.getOutput());
output.shouldContain("gc is too frequent");
output = triggerJcmd("GC.elastic_heap", "young_commit_percent=40");
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: young generation commit percent 40, uncommitted memory 251658240 B]");
for (int i = 0; i < 1000 * 3; i++) {
arr = new byte[200*1024];
arr = new byte[200*1024];
Thread.sleep(1);
}
output = triggerJcmd("GC.elastic_heap", "young_commit_percent=0");
System.out.println(output.getOutput());
}
private static OutputAnalyzer triggerJcmd(String arg1, String arg2) throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
JDKToolLauncher jcmd = JDKToolLauncher.create("jcmd")
.addToolArg(pid);
if (arg1 != null) {
jcmd.addToolArg(arg1);
}
if (arg2 != null) {
jcmd.addToolArg(arg2);
}
ProcessBuilder pb = new ProcessBuilder(jcmd.getCommand());
return new OutputAnalyzer(pb.start());
}
private static int getRss() throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
int rss = 0;
Process ps = Runtime.getRuntime().exec("cat /proc/"+pid+"/status");
ps.waitFor();
BufferedReader br = new BufferedReader(new InputStreamReader(ps.getInputStream()));
String line;
while (( line = br.readLine()) != null ) {
if (line.startsWith("VmRSS:") ) {
int numEnd = line.length() - 3;
int numBegin = line.lastIndexOf(" ", numEnd - 1) + 1;
rss = Integer.parseInt(line.substring(numBegin, numEnd));
break;
}
}
return rss;
}
}
/*
* Copyright (c) 2019 Alibaba Group Holding Limited. All Rights Reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Alibaba designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*/
import java.io.BufferedReader;
import java.io.InputStreamReader;
import java.io.File;
import com.oracle.java.testlibrary.*;
/* @test
* @summary test elastic-heap with jcmd
* @library /testlibrary
* @build TestElasticHeapGenerationLimitUncommitIhop
* @run main/othervm/timeout=200 -XX:+UseG1GC -XX:+G1ElasticHeap -Xmx1000m -Xms1000m -XX:MaxNewSize=400m -XX:G1HeapRegionSize=1m -XX:ElasticHeapEagerMixedGCIntervalMillis=1000 -XX:ElasticHeapInitialMarkIntervalMinMillis=0 -XX:+AlwaysPreTouch -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:+PrintGCDateStamps -XX:ElasticHeapYGCIntervalMinMillis=10 TestElasticHeapGenerationLimitUncommitIhop
*/
public class TestElasticHeapGenerationLimitUncommitIhop {
public static void main(String[] args) throws Exception {
byte[] arr = new byte[200*1024];
OutputAnalyzer output;
// Promote 480M into old gen
Object[] root = new Object[5 * 1024];
for (int i = 0; i < 1000 * 5 - 200; i++) {
root[i] = new byte[100*1024];
Thread.sleep(1);
}
System.gc();
int rssFull = getRss();
System.out.println("Full rss: " + rssFull);
root = null;
output = triggerJcmd("GC.elastic_heap", "uncommit_ihop=30", "young_commit_percent=50");
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: young generation commit percent 50, uncommitted memory 209715200 B]");
output.shouldHaveExitValue(0);
// Guaranteed gc will happen and mixed gc will finish
for (int i = 0; i < 1000 * 10; i++) {
arr = new byte[200];
Thread.sleep(1);
}
int rssLess = getRss();
System.out.println("Less rss: " + rssLess);
Asserts.assertTrue(rssLess < rssFull);
Asserts.assertTrue(Math.abs(rssFull - rssLess) > 350 * 1024);
}
private static OutputAnalyzer triggerJcmd(String arg1, String arg2, String arg3) throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
JDKToolLauncher jcmd = JDKToolLauncher.create("jcmd")
.addToolArg(pid);
if (arg1 != null) {
jcmd.addToolArg(arg1);
}
if (arg2 != null) {
jcmd.addToolArg(arg2);
}
if (arg3 != null) {
jcmd.addToolArg(arg3);
}
ProcessBuilder pb = new ProcessBuilder(jcmd.getCommand());
return new OutputAnalyzer(pb.start());
}
private static int getRss() throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
int rss = 0;
Process ps = Runtime.getRuntime().exec("cat /proc/"+pid+"/status");
ps.waitFor();
BufferedReader br = new BufferedReader(new InputStreamReader(ps.getInputStream()));
String line;
while (( line = br.readLine()) != null ) {
if (line.startsWith("VmRSS:") ) {
int numEnd = line.length() - 3;
int numBegin = line.lastIndexOf(" ", numEnd - 1) + 1;
rss = Integer.parseInt(line.substring(numBegin, numEnd));
break;
}
}
return rss;
}
}
/*
* Copyright (c) 2019 Alibaba Group Holding Limited. All Rights Reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Alibaba designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*/
import com.oracle.java.testlibrary.*;
/* @test
* @summary test elastic-heap error with jcmd
* @library /testlibrary
* @build TestElasticHeapJcmdError
* @run main/othervm/timeout=100 TestElasticHeapJcmdError
*/
public class TestElasticHeapJcmdError {
public static void main(String[] args) throws Exception {
ProcessBuilder serverBuilder;
serverBuilder = ProcessTools.createJavaProcessBuilder("-XX:+UseG1GC",
"-Xmx1g", "-Xms1g",
"-Xmn100m", "-XX:G1HeapRegionSize=1m",
"-XX:ElasticHeapYGCIntervalMinMillis=50",
"-verbose:gc", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-Dtest.jdk=" + System.getProperty("test.jdk"),
Server.class.getName());
Process server = serverBuilder.start();
OutputAnalyzer output = new OutputAnalyzer(server);
System.out.println(output.getOutput());
output.shouldContain("Error: -XX:+G1ElasticHeap is not enabled");
Asserts.assertTrue(output.getExitValue() == 0);
}
private static class Server {
public static void main(String[] args) throws Exception {
byte[] arr = new byte[200*1024];
// Allocate 200k per 1ms, 200M per second
// so 2 GCs per second
for (int i = 0; i < 1000 * 5; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
triggerJcmd("GC.elastic_heap", "young_commit_percent=50");
for (int i = 0; i < 1000 * 5; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
}
private static void triggerJcmd(String arg1, String arg2) throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
JDKToolLauncher jcmd = JDKToolLauncher.create("jcmd")
.addToolArg(pid);
if (arg1 != null) {
jcmd.addToolArg(arg1);
}
if (arg2 != null) {
jcmd.addToolArg(arg2);
}
ProcessBuilder pb = new ProcessBuilder(jcmd.getCommand());
OutputAnalyzer output = new OutputAnalyzer(pb.start());
System.out.println(output.getOutput());
output.shouldHaveExitValue(0);
}
}
}
/*
* Copyright (c) 2019 Alibaba Group Holding Limited. All Rights Reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Alibaba designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*/
import java.io.BufferedReader;
import java.io.InputStreamReader;
import java.io.File;
import com.oracle.java.testlibrary.*;
/* @test
* @summary test elastic-heap with many jcmds
* @library /testlibrary
* @build TestElasticHeapManyJcmds
* @run main/othervm/timeout=300 TestElasticHeapManyJcmds
*/
class TestElasticHeapManyJcmdsSetter implements Runnable {
private int loopNum;
public TestElasticHeapManyJcmdsSetter(int ln) {
loopNum = ln;
}
private OutputAnalyzer triggerJcmd(String arg1, String arg2) throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
JDKToolLauncher jcmd = JDKToolLauncher.create("jcmd")
.addToolArg(pid);
if (arg1 != null) {
jcmd.addToolArg(arg1);
}
if (arg2 != null) {
jcmd.addToolArg(arg2);
}
ProcessBuilder pb = new ProcessBuilder(jcmd.getCommand());
return new OutputAnalyzer(pb.start());
}
public void run() {
OutputAnalyzer output;
for (int i = 0; i < loopNum; i++) {
int p = 50 + i % 50;
try {
output = triggerJcmd("GC.elastic_heap", "young_commit_percent=" + p);
System.out.println(output.getOutput());
} catch (Exception e) {
System.out.println("Error: "+ e.getMessage());
}
}
}
}
public class TestElasticHeapManyJcmds {
public static void main(String[] args) throws Exception {
ProcessBuilder serverBuilder;
if (Platform.isDebugBuild()) {
serverBuilder = ProcessTools.createJavaProcessBuilder(
"-XX:+UseG1GC",
"-XX:+G1ElasticHeap", "-Xmx1g", "-Xms1g",
"-Xmn100m", "-XX:G1HeapRegionSize=1m",
"-XX:ElasticHeapYGCIntervalMinMillis=50",
"-verbose:gc", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-Dtest.jdk=" + System.getProperty("test.jdk"),
Server.class.getName(), "10");
} else {
serverBuilder = ProcessTools.createJavaProcessBuilder(
"-XX:+UseG1GC",
"-XX:+G1ElasticHeap", "-Xmx1g", "-Xms1g",
"-Xmn100m", "-XX:G1HeapRegionSize=1m",
"-XX:ElasticHeapYGCIntervalMinMillis=50",
"-verbose:gc", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-Dtest.jdk=" + System.getProperty("test.jdk"),
Server.class.getName(), "30");
}
Process server = serverBuilder.start();
OutputAnalyzer output = new OutputAnalyzer(server);
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap");
Asserts.assertTrue(output.getExitValue() == 0);
}
private static class Server {
public static void main(String[] args) throws Exception {
int loopNum = Integer.parseInt(args[0]);
byte[] arr = new byte[200*1024];
for (int i = 0; i < 1000 * 5; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
TestElasticHeapManyJcmdsSetter setter = new TestElasticHeapManyJcmdsSetter(loopNum);
int count = Runtime.getRuntime().availableProcessors();
Thread[] t = new Thread[count];
for (int i = 0; i < count ; i++) {
t[i] = new Thread(setter, "t" + i);
t[i].start();
}
for (int i = 0; i < 1000 * 60; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
System.exit(0);
}
}
}
/*
* Copyright (c) 2019 Alibaba Group Holding Limited. All Rights Reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Alibaba designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*/
import com.oracle.java.testlibrary.*;
/* @test
* @summary test elastic-heap
* @library /testlibrary
* @build TestElasticHeapMisc
* @run main/othervm/timeout=100 TestElasticHeapMisc
*/
public class TestElasticHeapMisc {
public static void main(String[] args) throws Exception {
ProcessBuilder serverBuilder;
serverBuilder = ProcessTools.createJavaProcessBuilder("-XX:+UseG1GC",
"-XX:+G1ElasticHeap", "-Xmx1g", "-Xms1g",
"-Xmn100m", "-XX:G1HeapRegionSize=1m", "-XX:SurvivorRatio=1",
"-XX:ElasticHeapYGCIntervalMinMillis=50",
"-XX:InitiatingHeapOccupancyPercent=80",
"-verbose:gc", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-Dtest.jdk=" + System.getProperty("test.jdk"),
GenerationLimitLargerSurvivor.class.getName());
Process server = serverBuilder.start();
OutputAnalyzer output = new OutputAnalyzer(server);
System.out.println(output.getOutput());
// Will uncommit 70M in first time
output.shouldContain("[Elastic Heap concurrent thread: uncommit 71680K");
Asserts.assertTrue(output.getExitValue() == 0);
serverBuilder = ProcessTools.createJavaProcessBuilder("-XX:+UseG1GC",
"-XX:+G1ElasticHeap", "-Xmx1g", "-Xms1g",
"-Xmn100m", "-XX:G1HeapRegionSize=1m",
"-XX:ElasticHeapYGCIntervalMinMillis=50",
"-XX:ElasticHeapPeriodicYGCIntervalMillis=200",
"-XX:ElasticHeapPeriodicInitialMarkIntervalMillis=5000",
"-XX:InitiatingHeapOccupancyPercent=80",
"-verbose:gc", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-Dtest.jdk=" + System.getProperty("test.jdk"),
NoElasticHeapGC.class.getName());
server = serverBuilder.start();
output = new OutputAnalyzer(server);
System.out.println(output.getOutput());
output.shouldNotContain("Elastic Heap triggered GC");
Asserts.assertTrue(output.getExitValue() == 0);
}
private static class NoElasticHeapGC {
public static void main(String[] args) throws Exception {
byte[] arr;
for (int i = 0; i < 1000 * 15; i++) {
arr = new byte[20*1024];
Thread.sleep(1);
}
}
}
private static class GenerationLimitLargerSurvivor {
public static void main(String[] args) throws Exception {
Object[] root = new Object[1024*1024];
int rootIndex = 0;
byte[] arr = new byte[200*1024];
// Allocate 200k per 1ms , 200M per second
// so 2 GCs per second
// Make 10% survive
for (int i = 0; i < 1000 * 5; i++) {
if (i % 10 != 0) {
arr = new byte[200*1024];
}
else {
root[rootIndex++] = new byte[200*1024];
}
Thread.sleep(1);
}
triggerJcmd("GC.elastic_heap", "young_commit_percent=30");
for (int i = 0; i < 1000 * 7; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
}
private static void triggerJcmd(String arg1, String arg2) throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
JDKToolLauncher jcmd = JDKToolLauncher.create("jcmd")
.addToolArg(pid);
if (arg1 != null) {
jcmd.addToolArg(arg1);
}
if (arg2 != null) {
jcmd.addToolArg(arg2);
}
ProcessBuilder pb = new ProcessBuilder(jcmd.getCommand());
OutputAnalyzer output = new OutputAnalyzer(pb.start());
System.out.println(output.getOutput());
output.shouldHaveExitValue(0);
}
}
}
/*
* Copyright (c) 2019 Alibaba Group Holding Limited. All Rights Reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Alibaba designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*/
import java.io.BufferedReader;
import java.io.InputStreamReader;
import java.io.File;
import com.oracle.java.testlibrary.*;
/* @test
* @summary test elastic-heap periodic gc djust
* @library /testlibrary
* @build TestElasticHeapPeriodic
* @run main/othervm/timeout=600
-XX:+UseG1GC -XX:+G1ElasticHeap -Xmx1000m -Xms1000m
-XX:ElasticHeapPeriodicYGCIntervalMillis=400
-XX:ElasticHeapPeriodicUncommitStartupDelay=0
-Xmn200m -XX:G1HeapRegionSize=1m -XX:+AlwaysPreTouch
-XX:ElasticHeapYGCIntervalMinMillis=50
-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps
TestElasticHeapPeriodic
*/
public class TestElasticHeapPeriodic {
public static void main(String[] args) throws Exception {
for (int i = 0; i < 3; i++) {
test();
}
}
public static void test() throws Exception {
OutputAnalyzer output;
byte[] arr = new byte[200*1024];
// Allocate 200k per 1ms , 200M per second
// so 2 GCs per second
for (int i = 0; i < 1000 * 3; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
int rssFull = getRss();
System.out.println("rssFull: " + rssFull);
output = triggerJcmd("GC.elastic_heap", null);
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: inactive]");
output.shouldHaveExitValue(0);
output = triggerJinfo("+ElasticHeapPeriodicUncommit");
output.shouldHaveExitValue(0);
output = triggerJinfo("ElasticHeapPeriodicMinYoungCommitPercent=50");
output.shouldHaveExitValue(0);
// Allocate 200k per ms , 200M per second
// so 1 GC per second
for (int i = 0; i < 1000 * 12; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
int rss50 = getRss();
System.out.println("rss50: " + rss50);
output = triggerJcmd("GC.elastic_heap", null);
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: young generation commit percent 50, uncommitted memory 104857600 B]");
output.shouldHaveExitValue(0);
System.gc();
Asserts.assertTrue(rss50 < rssFull);
Asserts.assertTrue(Math.abs(rssFull - rss50) > 80 * 1024);
// Allocate 1200k per ms, 1200M per second
// 6 GCs per second
for (int i = 0; i < 1000 * 10; i++) {
arr = new byte[400*1024];
arr = new byte[400*1024];
arr = new byte[400*1024];
Thread.sleep(1);
}
int rss100 = getRss();
System.out.println("rss100: " + rss100);
output = triggerJcmd("GC.elastic_heap", null);
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: young generation commit percent 100, uncommitted memory 0 B]");
Asserts.assertTrue(rss50 < rssFull);
Asserts.assertTrue(Math.abs(rssFull - rss50) > 80 * 1024);
output.shouldHaveExitValue(0);
output = triggerJinfo("ElasticHeapPeriodicMinYoungCommitPercent=30");
output.shouldHaveExitValue(0);
// Allocate 100k per ms , 100M per second
// so 0.5 GC per second
for (int i = 0; i < 1000 * 22; i++) {
arr = new byte[100*1024];
Thread.sleep(1);
}
// Auto adjust happens
output = triggerJcmd("GC.elastic_heap", null);
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: young generation commit percent 30, uncommitted memory 146800640 B]");
output.shouldHaveExitValue(0);
System.gc();
int rss30 = getRss();
System.out.println("rss30: " + rss30);
Asserts.assertTrue(rss30 < rss50);
Asserts.assertTrue(Math.abs(rssFull - rss30) > 120 * 1024);
output = triggerJinfo("ElasticHeapPeriodicMinYoungCommitPercent=70");
output.shouldHaveExitValue(0);
for (int i = 0; i < 1000 * 3; i++) {
arr = new byte[200*1024];
arr = new byte[200*1024];
arr = new byte[200*1024];
Thread.sleep(1);
}
// Allocate 200k per ms , 200M per second
// so 1 GC per second
for (int i = 0; i < 1000 * 12; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
int rss70 = getRss();
System.out.println("rss70: " + rss70);
output = triggerJcmd("GC.elastic_heap", null);
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: young generation commit percent 70, uncommitted memory 62914560 B]");
output.shouldHaveExitValue(0);
Asserts.assertTrue(rss70 < rssFull);
Asserts.assertTrue(rss70 > rss50);
Asserts.assertTrue(Math.abs(rssFull - rss70) > 40 * 1024);
output = triggerJinfo("-ElasticHeapPeriodicUncommit");
output.shouldHaveExitValue(0);
for (int i = 0; i < 1000 * 3; i++) {
arr = new byte[200*1024];
arr = new byte[200*1024];
arr = new byte[200*1024];
arr = new byte[200*1024];
arr = new byte[200*1024];
Thread.sleep(1);
}
}
private static OutputAnalyzer triggerJcmd(String arg1, String arg2) throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
JDKToolLauncher jcmd = JDKToolLauncher.create("jcmd")
.addToolArg(pid);
if (arg1 != null) {
jcmd.addToolArg(arg1);
}
if (arg2 != null) {
jcmd.addToolArg(arg2);
}
ProcessBuilder pb = new ProcessBuilder(jcmd.getCommand());
return new OutputAnalyzer(pb.start());
}
private static OutputAnalyzer triggerJinfo(String arg) throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
JDKToolLauncher jcmd = JDKToolLauncher.create("jinfo")
.addToolArg("-flag")
.addToolArg(arg)
.addToolArg(pid);
ProcessBuilder pb = new ProcessBuilder(jcmd.getCommand());
return new OutputAnalyzer(pb.start());
}
private static int getRss() throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
int rss = 0;
Process ps = Runtime.getRuntime().exec("cat /proc/"+pid+"/status");
ps.waitFor();
BufferedReader br = new BufferedReader(new InputStreamReader(ps.getInputStream()));
String line;
while (( line = br.readLine()) != null ) {
if (line.startsWith("VmRSS:") ) {
int numEnd = line.length() - 3;
int numBegin = line.lastIndexOf(" ", numEnd - 1) + 1;
rss = Integer.parseInt(line.substring(numBegin, numEnd));
break;
}
}
return rss;
}
}
/*
* Copyright (c) 2019 Alibaba Group Holding Limited. All Rights Reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Alibaba designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*/
import com.oracle.java.testlibrary.*;
/* @test
* @summary test elastic-heap periodic adjust with humongous allocation
* @library /testlibrary
* @build TestElasticHeapPeriodicHumAlloc
* @run main/othervm/timeout=300 TestElasticHeapPeriodicHumAlloc
*/
public class TestElasticHeapPeriodicHumAlloc {
public static void main(String[] args) throws Exception {
ProcessBuilder serverBuilder;
serverBuilder = ProcessTools.createJavaProcessBuilder(
"-XX:ElasticHeapPeriodicInitialMarkIntervalMillis=60000",
"-XX:ElasticHeapPeriodicMinYoungCommitPercent=10",
"-XX:+ElasticHeapPeriodicUncommit",
"-XX:ElasticHeapPeriodicUncommitStartupDelay=1",
"-XX:ElasticHeapPeriodicYGCIntervalMillis=5000",
"-XX:ElasticHeapYGCIntervalMinMillis=2000",
"-Xmx1g", "-Xms1g", "-XX:MaxNewSize=350m", "-XX:+PrintGC",
"-XX:+PrintGCDateStamps", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-XX:+UseG1GC", "-XX:+G1ElasticHeap",
Server.class.getName());
Process server = serverBuilder.start();
OutputAnalyzer output = new OutputAnalyzer(server);
System.out.println(output.getOutput());
output.shouldContain("G1 Humongous Allocation");
Asserts.assertTrue(output.getExitValue() == 0);
}
private static class Server {
public static void main(String[] args) throws Exception {
for (int i = 0; i < 2; i++) {
testloop();
}
}
public static void testloop() throws Exception {
Object[] root = new Object[40 * 1024];
for (int i = 0; i < 40 * 1024; i++) {
root[i] = new byte[10*1024];
if (i % 10 == 0) {
Thread.sleep(1);
}
}
byte[] array;
for (int i = 0; i < 1000 * 20; i++) {
array = new byte[2*1024];
if ((i % 10)== 0) {
array = new byte[600*1024];
}
Thread.sleep(1);
}
for (int i = 0; i < 1000 * 20; i++) {
array = new byte[100*1024];
array = new byte[100*1024];
array = new byte[100*1024];
if ((i % 10)== 0) {
array = new byte[600*1024];
}
Thread.sleep(1);
}
root = null;
for (int i = 0; i < 1000 * 20; i++) {
array = new byte[2*1024];
if ((i % 10)== 0) {
array = new byte[600*1024];
}
Thread.sleep(1);
}
for (int i = 0; i < 1000 * 20; i++) {
array = new byte[100*1024];
array = new byte[100*1024];
array = new byte[100*1024];
if ((i % 10)== 0) {
array = new byte[600*1024];
}
Thread.sleep(1);
}
}
}
}
/*
* Copyright (c) 2019 Alibaba Group Holding Limited. All Rights Reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Alibaba designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*/
import java.io.BufferedReader;
import java.io.InputStreamReader;
import java.io.File;
import com.oracle.java.testlibrary.*;
/* @test
* @summary test elastic-heap periodic old gc uncommit with RSS check
* @library /testlibrary /testlibrary/whitebox
* @build TestElasticHeapPeriodicInitialMark
* @run main/othervm/timeout=200 TestElasticHeapPeriodicInitialMark
*/
public class TestElasticHeapPeriodicInitialMark {
public static void main(String[] args) throws Exception {
ProcessBuilder serverBuilder;
serverBuilder = ProcessTools.createJavaProcessBuilder("-XX:+UseG1GC",
"-XX:+G1ElasticHeap", "-XX:+ElasticHeapPeriodicUncommit",
"-XX:ElasticHeapPeriodicInitialMarkIntervalMillis=5000",
"-XX:ElasticHeapEagerMixedGCIntervalMillis=1000",
"-XX:ElasticHeapPeriodicMinYoungCommitPercent=100",
"-XX:ElasticHeapPeriodicUncommitStartupDelay=0",
"-Xmx1g", "-Xms1g", "-Xmn200m", "-XX:G1HeapRegionSize=1m",
"-XX:ElasticHeapYGCIntervalMinMillis=50",
"-XX:ElasticHeapInitialMarkIntervalMinMillis=0",
"-verbose:gc", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-XX:+PrintGCDateStamps",
"-Dtest.jdk=" + System.getProperty("test.jdk"),
Server.class.getName());
Process server = serverBuilder.start();
OutputAnalyzer output = new OutputAnalyzer(server);
System.out.println(output.getOutput());
Asserts.assertTrue(output.getExitValue() == 0);
serverBuilder = ProcessTools.createJavaProcessBuilder("-XX:+UseG1GC",
"-XX:+G1ElasticHeap", "-XX:+ElasticHeapPeriodicUncommit",
"-XX:ElasticHeapPeriodicInitialMarkIntervalMillis=5000",
"-XX:ElasticHeapPeriodicMinYoungCommitPercent=100",
"-XX:ElasticHeapPeriodicUncommitStartupDelay=0",
"-Xmx1g", "-Xms1g", "-Xmn200m", "-XX:G1HeapRegionSize=1m",
"-XX:ElasticHeapYGCIntervalMinMillis=50",
"-XX:ElasticHeapInitialMarkIntervalMinMillis=0",
"-verbose:gc", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-XX:+PrintGCDateStamps",
"-Dtest.jdk=" + System.getProperty("test.jdk"),
Server2.class.getName());
server = serverBuilder.start();
output = new OutputAnalyzer(server);
output.shouldContain("initial-mark");
output.shouldNotContain("(Elastic Heap triggered GC) (young),");
output.shouldMatch("(Elastic Heap triggered GC).*initial-mark");
System.out.println(output.getOutput());
Asserts.assertTrue(output.getExitValue() == 0);
}
private static class Server2 {
public static void main(String[] args) throws Exception {
byte[] arr = new byte[200*1024];
// Guaranteed gc will happen and mixed gc will finish
for (int i = 0; i < 1000 * 20; i++) {
arr = new byte[100 * 1024];
arr = new byte[100 * 1024];
Thread.sleep(1);
}
}
}
private static class Server {
public static void main(String[] args) throws Exception {
// Pretouch young gen
byte[] arr = new byte[200*1024];
for (int i = 0; i < 1000 * 5; i++) {
arr = new byte[200*1024];
}
// Promote 500M into old gen
Object[] root = new Object[5 * 1024];
for (int i = 0; i < 5 * 1024; i++) {
root[i] = new byte[100*1024];
}
System.gc();
root = null;
int fullRss = getRss();
// Guaranteed gc will happen and mixed gc will finish
for (int i = 0; i < 1000 * 15; i++) {
arr = new byte[2*1024];
Thread.sleep(1);
}
int lessRss = getRss();
Asserts.assertTrue((fullRss > lessRss) && (Math.abs(fullRss - lessRss) > 300 * 1024));
}
private static int getRss() throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
int rss = 0;
Process ps = Runtime.getRuntime().exec("cat /proc/"+pid+"/status");
ps.waitFor();
BufferedReader br = new BufferedReader(new InputStreamReader(ps.getInputStream()));
String line;
while (( line = br.readLine()) != null ) {
if (line.startsWith("VmRSS:") ) {
int numEnd = line.length() - 3;
int numBegin = line.lastIndexOf(" ", numEnd - 1) + 1;
rss = Integer.parseInt(line.substring(numBegin, numEnd));
break;
}
}
return rss;
}
}
}
/*
* Copyright (c) 2019 Alibaba Group Holding Limited. All Rights Reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Alibaba designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*/
import com.oracle.java.testlibrary.*;
/* @test
* @summary test elastic-heap periodic adjust with OOM
* @library /testlibrary
* @build TestElasticHeapPeriodicOOM
* @run main/othervm/timeout=100 TestElasticHeapPeriodicOOM
*/
public class TestElasticHeapPeriodicOOM {
public static void main(String[] args) throws Exception {
ProcessBuilder serverBuilder;
serverBuilder = ProcessTools.createJavaProcessBuilder("-XX:+UseG1GC",
"-XX:+G1ElasticHeap", "-Xmx1g", "-Xms1g",
"-XX:+ElasticHeapPeriodicUncommit", "-XX:ElasticHeapPeriodicYGCIntervalMillis=50",
"-XX:ElasticHeapPeriodicUncommitStartupDelay=5",
"-Xmn100m", "-XX:G1HeapRegionSize=1m",
"-XX:ElasticHeapYGCIntervalMinMillis=10",
"-XX:ElasticHeapPeriodicMinYoungCommitPercent=30",
"-XX:InitiatingHeapOccupancyPercent=80",
"-verbose:gc", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-Dtest.jdk=" + System.getProperty("test.jdk"),
Server.class.getName());
Process server = serverBuilder.start();
OutputAnalyzer output = new OutputAnalyzer(server);
System.out.println(output.getOutput());
output.shouldContain("Elastic Heap concurrent thread: uncommit");
output.shouldContain("Elastic Heap recovers");
Asserts.assertTrue(output.getExitValue() != 0);
}
private static class Server {
public static void main(String[] args) throws Exception {
Object[] root = new Object[1024*1024];
int rootIndex = 0;
byte[] arr = new byte[200*1024];
// Allocate 200k per 1ms , 200M per second
// so 2 GCs per second
// Make 10% survive
for (int i = 0; i < 1000 * 10; i++) {
if (i % 10 != 0) {
arr = new byte[200*1024];
}
else {
root[rootIndex++] = new byte[200*1024];
}
Thread.sleep(1);
}
// Make 100M per second survive
for (int i = 0; i < 1000 * 10; i++) {
if (i % 2 != 0) {
arr = new byte[200*1024];
}
else {
root[rootIndex++] = new byte[200*1024];
}
Thread.sleep(1);
}
}
}
}
/*
* Copyright (c) 2019 Alibaba Group Holding Limited. All Rights Reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Alibaba designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*/
import com.oracle.java.testlibrary.*;
/* @test
* @summary test elastic-heap periodic adjust young gen
* @library /testlibrary
* @build TestElasticHeapPeriodicYoung
* @run main/othervm/timeout=200 TestElasticHeapPeriodicYoung
*/
public class TestElasticHeapPeriodicYoung {
public static void main(String[] args) throws Exception {
ProcessBuilder serverBuilder;
serverBuilder = ProcessTools.createJavaProcessBuilder("-XX:+UseG1GC",
"-XX:+G1ElasticHeap", "-Xmx1g", "-Xms1g",
"-XX:+ElasticHeapPeriodicUncommit", "-XX:ElasticHeapPeriodicYGCIntervalMillis=150",
"-XX:ElasticHeapPeriodicUncommitStartupDelay=5",
"-Xmn100m", "-XX:G1HeapRegionSize=1m", "-XX:SurvivorRatio=1",
"-XX:ElasticHeapYGCIntervalMinMillis=10",
"-XX:ElasticHeapPeriodicMinYoungCommitPercent=40",
"-XX:InitiatingHeapOccupancyPercent=80",
"-verbose:gc", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-Dtest.jdk=" + System.getProperty("test.jdk"),
Server.class.getName());
Process server = serverBuilder.start();
OutputAnalyzer output = new OutputAnalyzer(server);
System.out.println(output.getOutput());
// Will not deallocate 60M in first time because of 50M survivor size
String s = output.firstMatch("Elastic Heap concurrent thread: uncommit \\d+");
Asserts.assertTrue(s != null);
Asserts.assertFalse("Elastic Heap concurrent thread: uncommit 60".equals(s));
Asserts.assertTrue(output.getExitValue() == 0);
serverBuilder = ProcessTools.createJavaProcessBuilder("-XX:+UseG1GC",
"-XX:+G1ElasticHeap", "-Xmx1g", "-Xms1g",
"-XX:+ElasticHeapPeriodicUncommit", "-XX:ElasticHeapPeriodicYGCIntervalMillis=500",
"-XX:ElasticHeapPeriodicUncommitStartupDelay=1",
"-Xmn100m", "-XX:G1HeapRegionSize=1m",
"-XX:ElasticHeapYGCIntervalMinMillis=10",
"-XX:ElasticHeapPeriodicMinYoungCommitPercent=40",
"-XX:InitiatingHeapOccupancyPercent=80",
"-verbose:gc", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-Dtest.jdk=" + System.getProperty("test.jdk"),
Server2.class.getName());
server = serverBuilder.start();
output = new OutputAnalyzer(server);
System.out.println(output.getOutput());
output.shouldContain("Elastic Heap concurrent thread: uncommit");
Asserts.assertTrue(output.getExitValue() == 0);
}
private static class Server2 {
public static void main(String[] args) throws Exception {
byte[] arr = new byte[200*1024];
for (int i = 0; i < 1000 * 4; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
for (int i = 0; i < 1000 * 20; i++) {
Thread.sleep(1);
}
}
}
private static class Server {
public static void main(String[] args) throws Exception {
Object[] root = new Object[1024*1024];
int rootIndex = 0;
byte[] arr = new byte[200*1024];
// Allocate 200k per 1ms , 200M per second
// so 2 GCs per second
// Make 10% survive
for (int i = 0; i < 1000 * 12; i++) {
if (i % 10 != 0) {
arr = new byte[200*1024];
}
else {
root[rootIndex++] = new byte[200*1024];
}
Thread.sleep(1);
}
}
}
}
/*
* Copyright (c) 2019 Alibaba Group Holding Limited. All Rights Reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Alibaba designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*/
import java.io.BufferedReader;
import java.io.InputStreamReader;
import java.io.File;
import com.oracle.java.testlibrary.*;
/* @test
* @summary test elastic-heap setting softmx percent with RSS check
* @library /testlibrary /testlibrary/whitebox
* @build TestElasticHeapSoftmxPercent
* @run main/othervm/timeout=200 TestElasticHeapSoftmxPercent
*/
public class TestElasticHeapSoftmxPercent {
public static void main(String[] args) throws Exception {
ProcessBuilder serverBuilder;
serverBuilder = ProcessTools.createJavaProcessBuilder("-XX:+UseG1GC",
"-XX:+G1ElasticHeap",
"-Xmx1000m", "-Xms1000m", "-XX:G1HeapRegionSize=1m", "-XX:+AlwaysPreTouch",
"-verbose:gc", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-Dtest.jdk=" + System.getProperty("test.jdk"),
Server.class.getName());
Process server = serverBuilder.start();
OutputAnalyzer output = new OutputAnalyzer(server);
System.out.println(output.getOutput());
// Should not have regular initial-mark
output.shouldNotContain("[GC pause (G1 Evacuation Pause) (young) (initial-mark)");
Asserts.assertTrue(output.getExitValue() == 0);
}
private static class Server {
public static void main(String[] args) throws Exception {
// Promote 400M into old gen
Object[] root = new Object[4 * 1024];
for (int i = 0; i < 4 * 1024; i++) {
root[i] = new byte[100*1024];
}
byte[] arr = new byte[200*1024];
// Allocate 200k per 1ms, 200M per second
for (int i = 0; i < 1000 * 10; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
System.gc();
int fullRss = getRss();
System.out.println("Full rss: " + fullRss);
Asserts.assertTrue(fullRss > 1000 * 1024);
OutputAnalyzer output = triggerJcmd("GC.elastic_heap", "softmx_percent=50");
System.out.println(output.getOutput());
// Allocate 400k per 1ms, 400M per second
for (int i = 0; i < 1000 * 10; i++) {
arr = new byte[400*1024];
Thread.sleep(1);
}
output = triggerJcmd("GC.elastic_heap", null);
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: inactive]");
output = triggerJcmd("GC.elastic_heap", "softmx_percent=60");
System.out.println(output.getOutput());
// Allocate 400k per 1ms, 400M per second
for (int i = 0; i < 1000 * 10; i++) {
arr = new byte[400*1024];
Thread.sleep(1);
}
output = triggerJcmd("GC.elastic_heap", null);
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: in softmx mode]");
output.shouldContain("[GC.elastic_heap: softmx percent 60, uncommitted memory 419430400 B]");
int rss60 = getRss();
System.out.println("60% rss: " + rss60);
Asserts.assertTrue(Math.abs(fullRss - rss60) > 350 * 1024);
System.gc();
output = triggerJcmd("GC.elastic_heap", "softmx_percent=80");
System.out.println(output.getOutput());
// Allocate 400k per 1ms, 400M per second
for (int i = 0; i < 1000 * 10; i++) {
arr = new byte[400*1024];
Thread.sleep(1);
}
output = triggerJcmd("GC.elastic_heap", null);
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: in softmx mode]");
output.shouldContain("[GC.elastic_heap: softmx percent 80, uncommitted memory 209715200 B]");
int rss80 = getRss();
System.out.println("80% rss: " + rss80);
Asserts.assertTrue(Math.abs(fullRss - rss80) > 150 * 1024);
Asserts.assertTrue(Math.abs(rss80 - rss60) > 150 * 1024);
output = triggerJcmd("GC.elastic_heap", "softmx_percent=0");
System.out.println(output.getOutput());
// Allocate 400k per 1ms, 400M per second
for (int i = 0; i < 1000 * 10; i++) {
arr = new byte[400*1024];
Thread.sleep(1);
}
output = triggerJcmd("GC.elastic_heap", null);
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: inactive]");
int rss100 = getRss();
System.out.println("100% rss: " + rss100);
Asserts.assertTrue(Math.abs(fullRss - rss100) < 100 * 1024);
output = triggerJcmd("GC.elastic_heap", "softmx_percent=70");
System.out.println(output.getOutput());
// Allocate 400k per 1ms, 400M per second
for (int i = 0; i < 1000 * 10; i++) {
arr = new byte[400*1024];
Thread.sleep(1);
}
System.gc();
output = triggerJcmd("GC.elastic_heap", null);
System.out.println(output.getOutput());
output.shouldContain("[GC.elastic_heap: in softmx mode]");
output.shouldContain("[GC.elastic_heap: softmx percent 70, uncommitted memory 314572800 B]");
int rss70 = getRss();
System.out.println("70% rss: " + rss70);
Asserts.assertTrue(Math.abs(fullRss - rss70) > 250 * 1024);
}
private static OutputAnalyzer triggerJcmd(String arg1, String arg2) throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
JDKToolLauncher jcmd = JDKToolLauncher.create("jcmd")
.addToolArg(pid);
if (arg1 != null) {
jcmd.addToolArg(arg1);
}
if (arg2 != null) {
jcmd.addToolArg(arg2);
}
ProcessBuilder pb = new ProcessBuilder(jcmd.getCommand());
return new OutputAnalyzer(pb.start());
}
private static int getRss() throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
int rss = 0;
Process ps = Runtime.getRuntime().exec("cat /proc/"+pid+"/status");
ps.waitFor();
BufferedReader br = new BufferedReader(new InputStreamReader(ps.getInputStream()));
String line;
while (( line = br.readLine()) != null ) {
if (line.startsWith("VmRSS:") ) {
int numEnd = line.length() - 3;
int numBegin = line.lastIndexOf(" ", numEnd - 1) + 1;
rss = Integer.parseInt(line.substring(numBegin, numEnd));
break;
}
}
return rss;
}
}
}
/*
* Copyright (c) 2019 Alibaba Group Holding Limited. All Rights Reserved.
* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
*
* This code is free software; you can redistribute it and/or modify it
* under the terms of the GNU General Public License version 2 only, as
* published by the Free Software Foundation. Alibaba designates this
* particular file as subject to the "Classpath" exception as provided
* by Oracle in the LICENSE file that accompanied this code.
*
* This code is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License
* version 2 for more details (a copy is included in the LICENSE file that
* accompanied this code).
*
* You should have received a copy of the GNU General Public License version
* 2 along with this work; if not, write to the Free Software Foundation,
* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA.
*/
import java.io.BufferedReader;
import java.io.InputStreamReader;
import java.io.File;
import com.oracle.java.testlibrary.*;
/* @test
* @summary test elastic-heap setting heap commit percent fail
* @library /testlibrary /testlibrary/whitebox
* @build TestElasticHeapSoftmxPercentFail
* @run main/othervm/timeout=200 TestElasticHeapSoftmxPercentFail
*/
public class TestElasticHeapSoftmxPercentFail {
public static void main(String[] args) throws Exception {
ProcessBuilder serverBuilder;
Process server;
OutputAnalyzer output;
serverBuilder = ProcessTools.createJavaProcessBuilder("-XX:+UseG1GC",
"-XX:+G1ElasticHeap",
"-Xmx1000m", "-Xms1000m", "-XX:G1HeapRegionSize=1m", "-XX:+AlwaysPreTouch",
"-verbose:gc", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-Dtest.jdk=" + System.getProperty("test.jdk"),
Server.class.getName());
server = serverBuilder.start();
output = new OutputAnalyzer(server);
System.out.println(output.getOutput());
output.shouldContain("softmx percent setting failed");
Asserts.assertTrue(output.getExitValue() == 0);
serverBuilder = ProcessTools.createJavaProcessBuilder("-XX:+UseG1GC",
"-XX:+G1ElasticHeap",
"-Xmx600m", "-Xms600m", "-XX:G1HeapRegionSize=1m", "-XX:+AlwaysPreTouch",
"-Xmn300m",
"-verbose:gc", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-Dtest.jdk=" + System.getProperty("test.jdk"),
Server2.class.getName());
server = serverBuilder.start();
output = new OutputAnalyzer(server);
System.out.println(output.getOutput());
output.shouldContain("to-space exhausted");
Asserts.assertTrue(output.getExitValue() == 0);
serverBuilder = ProcessTools.createJavaProcessBuilder("-XX:+UseG1GC",
"-XX:+G1ElasticHeap",
"-Xmx1000m", "-Xms1000m", "-XX:G1HeapRegionSize=1m", "-XX:+AlwaysPreTouch",
"-Xmn300m",
"-verbose:gc", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-Dtest.jdk=" + System.getProperty("test.jdk"),
Server3.class.getName());
server = serverBuilder.start();
output = new OutputAnalyzer(server);
System.out.println(output.getOutput());
output.shouldNotContain("to-space exhausted");
output.shouldNotContain("softmx percent setting failed");
Asserts.assertTrue(output.getExitValue() == 0);
serverBuilder = ProcessTools.createJavaProcessBuilder("-XX:+UseG1GC",
"-XX:+G1ElasticHeap",
"-Xmx1000m", "-Xms1000m", "-XX:G1HeapRegionSize=1m", "-XX:+AlwaysPreTouch",
"-Xmn200m",
"-verbose:gc", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-Dtest.jdk=" + System.getProperty("test.jdk"),
Server4.class.getName());
server = serverBuilder.start();
output = new OutputAnalyzer(server);
System.out.println(output.getOutput());
output.shouldContain("Eden: 101376.0K");
output.shouldNotContain("softmx percent setting failed");
Asserts.assertTrue(output.getExitValue() == 0);
serverBuilder = ProcessTools.createJavaProcessBuilder("-XX:+UseG1GC",
"-XX:+G1ElasticHeap",
"-Xmx1000m", "-Xms500m", "-XX:G1HeapRegionSize=1m", "-XX:+AlwaysPreTouch",
"-Xmn200m",
"-verbose:gc", "-XX:+PrintGCDetails", "-XX:+PrintGCTimeStamps",
"-Dtest.jdk=" + System.getProperty("test.jdk"),
Server4.class.getName());
server = serverBuilder.start();
output = new OutputAnalyzer(server);
System.out.println(output.getOutput());
output.shouldContain("same to Xmx");
Asserts.assertTrue(output.getExitValue() != 0);
}
private static class Server {
public static void main(String[] args) throws Exception {
// Promote 400M into old gen
Object[] root = new Object[4 * 1024];
for (int i = 0; i < 4 * 1024; i++) {
root[i] = new byte[99*1024];
}
byte[] arr = new byte[200*1024];
// Allocate 200k per 1ms, 200M per second
for (int i = 0; i < 1000 * 10; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
OutputAnalyzer output = triggerJcmd("GC.elastic_heap", "softmx_percent=50");
System.out.println(output.getOutput());
// Allocate 200k per 1ms, 200M per second
for (int i = 0; i < 1000 * 10; i++) {
arr = new byte[200*1024];
Thread.sleep(1);
}
}
private static OutputAnalyzer triggerJcmd(String arg1, String arg2) throws Exception {
String pid = Integer.toString(ProcessTools.getProcessId());
JDKToolLauncher jcmd = JDKToolLauncher.create("jcmd")
.addToolArg(pid);
if (arg1 != null) {
jcmd.addToolArg(arg1);
}
if (arg2 != null) {
jcmd.addToolArg(arg2);
}
ProcessBuilder pb = new ProcessBuilder(jcmd.getCommand());
return new OutputAnalyzer(pb.start());
}
}
private static class Server2 extends Server {
public static void main(String[] args) throws Exception {
// Promote 400M into old gen
Object[] root = new Object[4 * 1024];
for (int i = 0; i < 4 * 1024; i++) {
root[i] = new byte[99*1024];
}
byte[] arr = new byte[200*1024];
for (int i = 0; i < 1000 * 10; i++) {
arr = new byte[200*1024];
arr = new byte[200*1024];
Thread.sleep(1);
}
}
}
private static class Server3 extends Server {
public static void main(String[] args) throws Exception {
// Promote 400M into old gen
Object[] root = new Object[4 * 1024];
for (int i = 0; i < 4 * 1024; i++) {
root[i] = new byte[99*1024];
}
byte[] arr = new byte[200*1024];
// Allocate 200k per 1ms, 200M per second
for (int i = 0; i < 1000 * 10; i++) {
arr = new byte[200*1024];
arr = new byte[200*1024];
Thread.sleep(1);
}
OutputAnalyzer output = Server.triggerJcmd("GC.elastic_heap", "softmx_percent=60");
System.out.println(output.getOutput());
// Allocate 200k per 1ms, 200M per second
for (int i = 0; i < 1000 * 10; i++) {
arr = new byte[200*1024];
arr = new byte[200*1024];
Thread.sleep(1);
}
}
}
private static class Server4 extends Server {
public static void main(String[] args) throws Exception {
// Promote 200M into old gen
Object[] root = new Object[2 * 1024];
for (int i = 0; i < 2 * 1024; i++) {
root[i] = new byte[99*1024];
}
byte[] arr = new byte[200*1024];
// Allocate 200k per 1ms, 200M per second
for (int i = 0; i < 1000 * 10; i++) {
arr = new byte[200*1024];
arr = new byte[200*1024];
Thread.sleep(1);
}
OutputAnalyzer output = Server.triggerJcmd("GC.elastic_heap", "softmx_percent=50");
System.out.println(output.getOutput());
// Allocate 200k per 1ms, 200M per second
for (int i = 0; i < 1000 * 10; i++) {
arr = new byte[200*1024];
arr = new byte[200*1024];
Thread.sleep(1);
}
}
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册