< prev index next > src/hotspot/share/compiler/compileBroker.cpp
Print this page
* questions.
*
*/
#include "precompiled.hpp"
+ #include "cds/aotLinkedClassBulkLoader.hpp"
+ #include "cds/cdsConfig.hpp"
#include "classfile/javaClasses.inline.hpp"
#include "classfile/symbolTable.hpp"
#include "classfile/vmClasses.hpp"
#include "classfile/vmSymbols.hpp"
#include "code/codeCache.hpp"
#include "code/codeHeapState.hpp"
#include "code/dependencyContext.hpp"
+ #include "code/SCCache.hpp"
#include "compiler/compilationLog.hpp"
#include "compiler/compilationMemoryStatistic.hpp"
#include "compiler/compilationPolicy.hpp"
#include "compiler/compileBroker.hpp"
+ #include "compiler/compilerDefinitions.inline.hpp"
#include "compiler/compileLog.hpp"
#include "compiler/compilerEvent.hpp"
#include "compiler/compilerOracle.hpp"
#include "compiler/directivesParser.hpp"
#include "gc/shared/memAllocator.hpp"
#include "runtime/os.hpp"
#include "runtime/perfData.hpp"
#include "runtime/safepointVerifiers.hpp"
#include "runtime/sharedRuntime.hpp"
#include "runtime/threads.hpp"
! #include "runtime/threadSMR.hpp"
#include "runtime/timerTrace.hpp"
#include "runtime/vframe.inline.hpp"
#include "utilities/debug.hpp"
#include "utilities/dtrace.hpp"
#include "utilities/events.hpp"
#include "utilities/formatBuffer.hpp"
#include "utilities/macros.hpp"
#ifdef COMPILER1
#include "c1/c1_Compiler.hpp"
#endif
#ifdef COMPILER2
#include "opto/c2compiler.hpp"
#include "runtime/os.hpp"
#include "runtime/perfData.hpp"
#include "runtime/safepointVerifiers.hpp"
#include "runtime/sharedRuntime.hpp"
#include "runtime/threads.hpp"
! #include "runtime/threadSMR.inline.hpp"
#include "runtime/timerTrace.hpp"
#include "runtime/vframe.inline.hpp"
+ #include "services/management.hpp"
#include "utilities/debug.hpp"
#include "utilities/dtrace.hpp"
#include "utilities/events.hpp"
#include "utilities/formatBuffer.hpp"
#include "utilities/macros.hpp"
+ #include "utilities/nonblockingQueue.inline.hpp"
#ifdef COMPILER1
#include "c1/c1_Compiler.hpp"
#endif
#ifdef COMPILER2
#include "opto/c2compiler.hpp"
#define DTRACE_METHOD_COMPILE_END_PROBE(method, comp_name, success)
#endif // ndef DTRACE_ENABLED
bool CompileBroker::_initialized = false;
volatile bool CompileBroker::_should_block = false;
volatile int CompileBroker::_print_compilation_warning = 0;
volatile jint CompileBroker::_should_compile_new_jobs = run_compilation;
// The installed compiler(s)
! AbstractCompiler* CompileBroker::_compilers[2];
// The maximum numbers of compiler threads to be determined during startup.
int CompileBroker::_c1_count = 0;
int CompileBroker::_c2_count = 0;
// An array of compiler names as Java String objects
jobject* CompileBroker::_compiler1_objects = nullptr;
jobject* CompileBroker::_compiler2_objects = nullptr;
CompileLog** CompileBroker::_compiler1_logs = nullptr;
CompileLog** CompileBroker::_compiler2_logs = nullptr;
// These counters are used to assign an unique ID to each compilation.
volatile jint CompileBroker::_compilation_id = 0;
volatile jint CompileBroker::_osr_compilation_id = 0;
volatile jint CompileBroker::_native_compilation_id = 0;
#define DTRACE_METHOD_COMPILE_END_PROBE(method, comp_name, success)
#endif // ndef DTRACE_ENABLED
bool CompileBroker::_initialized = false;
+ bool CompileBroker::_replay_initialized = false;
volatile bool CompileBroker::_should_block = false;
volatile int CompileBroker::_print_compilation_warning = 0;
volatile jint CompileBroker::_should_compile_new_jobs = run_compilation;
// The installed compiler(s)
! AbstractCompiler* CompileBroker::_compilers[3];
// The maximum numbers of compiler threads to be determined during startup.
int CompileBroker::_c1_count = 0;
int CompileBroker::_c2_count = 0;
+ int CompileBroker::_c3_count = 0;
+ int CompileBroker::_sc_count = 0;
// An array of compiler names as Java String objects
jobject* CompileBroker::_compiler1_objects = nullptr;
jobject* CompileBroker::_compiler2_objects = nullptr;
+ jobject* CompileBroker::_compiler3_objects = nullptr;
+ jobject* CompileBroker::_sc_objects = nullptr;
CompileLog** CompileBroker::_compiler1_logs = nullptr;
CompileLog** CompileBroker::_compiler2_logs = nullptr;
+ CompileLog** CompileBroker::_compiler3_logs = nullptr;
+ CompileLog** CompileBroker::_sc_logs = nullptr;
// These counters are used to assign an unique ID to each compilation.
volatile jint CompileBroker::_compilation_id = 0;
volatile jint CompileBroker::_osr_compilation_id = 0;
volatile jint CompileBroker::_native_compilation_id = 0;
elapsedTimer CompileBroker::_t_invalidated_compilation;
elapsedTimer CompileBroker::_t_bailedout_compilation;
uint CompileBroker::_total_bailout_count = 0;
uint CompileBroker::_total_invalidated_count = 0;
+ uint CompileBroker::_total_not_entrant_count = 0;
uint CompileBroker::_total_compile_count = 0;
uint CompileBroker::_total_osr_compile_count = 0;
uint CompileBroker::_total_standard_compile_count = 0;
uint CompileBroker::_total_compiler_stopped_count = 0;
uint CompileBroker::_total_compiler_restarted_count = 0;
uint CompileBroker::_sum_nmethod_code_size = 0;
jlong CompileBroker::_peak_compilation_time = 0;
CompilerStatistics CompileBroker::_stats_per_level[CompLevel_full_optimization];
+ CompilerStatistics CompileBroker::_scc_stats;
+ CompilerStatistics CompileBroker::_scc_stats_per_level[CompLevel_full_optimization + 1];
+ CompileQueue* CompileBroker::_c3_compile_queue = nullptr;
CompileQueue* CompileBroker::_c2_compile_queue = nullptr;
CompileQueue* CompileBroker::_c1_compile_queue = nullptr;
+ CompileQueue* CompileBroker::_sc1_compile_queue = nullptr;
+ CompileQueue* CompileBroker::_sc2_compile_queue = nullptr;
bool compileBroker_init() {
if (LogEvents) {
CompilationLog::init();
}
CompileTaskWrapper::~CompileTaskWrapper() {
CompilerThread* thread = CompilerThread::current();
CompileTask* task = thread->task();
CompileLog* log = thread->log();
if (log != nullptr && !task->is_unloaded()) task->log_task_done(log);
thread->set_task(nullptr);
thread->set_env(nullptr);
if (task->is_blocking()) {
bool free_task = false;
{
MutexLocker notifier(thread, task->lock());
task->mark_complete();
#if INCLUDE_JVMCI
! if (CompileBroker::compiler(task->comp_level())->is_jvmci()) {
if (!task->has_waiter()) {
// The waiting thread timed out and thus did not free the task.
free_task = true;
}
task->set_blocking_jvmci_compile_state(nullptr);
CompileTaskWrapper::~CompileTaskWrapper() {
CompilerThread* thread = CompilerThread::current();
CompileTask* task = thread->task();
CompileLog* log = thread->log();
+ AbstractCompiler* comp = thread->compiler();
if (log != nullptr && !task->is_unloaded()) task->log_task_done(log);
thread->set_task(nullptr);
thread->set_env(nullptr);
if (task->is_blocking()) {
bool free_task = false;
{
MutexLocker notifier(thread, task->lock());
task->mark_complete();
#if INCLUDE_JVMCI
! if (comp->is_jvmci()) {
if (!task->has_waiter()) {
// The waiting thread timed out and thus did not free the task.
free_task = true;
}
task->set_blocking_jvmci_compile_state(nullptr);
*/
bool CompileBroker::can_remove(CompilerThread *ct, bool do_it) {
assert(UseDynamicNumberOfCompilerThreads, "or shouldn't be here");
if (!ReduceNumberOfCompilerThreads) return false;
+ if (CompilationPolicy::have_recompilation_work()) return false;
+
AbstractCompiler *compiler = ct->compiler();
int compiler_count = compiler->num_compiler_threads();
bool c1 = compiler->is_c1();
// Keep at least 1 compiler thread of each type.
/**
* Add a CompileTask to a CompileQueue.
*/
void CompileQueue::add(CompileTask* task) {
! assert(MethodCompileQueue_lock->owned_by_self(), "must own lock");
task->set_next(nullptr);
task->set_prev(nullptr);
if (_last == nullptr) {
/**
* Add a CompileTask to a CompileQueue.
*/
void CompileQueue::add(CompileTask* task) {
! assert(_lock->owned_by_self(), "must own lock");
task->set_next(nullptr);
task->set_prev(nullptr);
if (_last == nullptr) {
}
// Mark the method as being in the compile queue.
task->method()->set_queued_for_compilation();
if (CIPrintCompileQueue) {
print_tty();
}
if (LogCompilation && xtty != nullptr) {
task->log_task_queued();
}
// Notify CompilerThreads that a task is available.
! MethodCompileQueue_lock->notify_all();
}
/**
* Empties compilation queue by putting all compilation tasks onto
* a freelist. Furthermore, the method wakes up all threads that are
* waiting on a compilation task to finish. This can happen if background
* compilation is disabled.
*/
void CompileQueue::free_all() {
! MutexLocker mu(MethodCompileQueue_lock);
CompileTask* next = _first;
// Iterate over all tasks in the compile queue
while (next != nullptr) {
CompileTask* current = next;
}
// Mark the method as being in the compile queue.
task->method()->set_queued_for_compilation();
+ task->mark_queued(os::elapsed_counter());
+
if (CIPrintCompileQueue) {
print_tty();
}
if (LogCompilation && xtty != nullptr) {
task->log_task_queued();
}
+ if (TrainingData::need_data() &&
+ !CDSConfig::is_dumping_final_static_archive()) { // FIXME: !!! MetaspaceShared::preload_and_dump() temporarily enables RecordTraining !!!
+ CompileTrainingData* tdata = CompileTrainingData::make(task);
+ if (tdata != nullptr) {
+ tdata->record_compilation_queued(task);
+ task->set_training_data(tdata);
+ }
+ }
+
// Notify CompilerThreads that a task is available.
! _lock->notify_all();
+ }
+
+ void CompileQueue::add_pending(CompileTask* task) {
+ assert(_lock->owned_by_self() == false, "must NOT own lock");
+ assert(UseLockFreeCompileQueues, "");
+ task->method()->set_queued_for_compilation();
+ _queue.push(*task);
+ // FIXME: additional coordination needed? e.g., is it possible for compiler thread to block w/o processing pending tasks?
+ if (is_empty()) {
+ MutexLocker ml(_lock);
+ _lock->notify_all();
+ }
+ }
+
+ static bool process_pending(CompileTask* task) {
+ // guarantee(task->method()->queued_for_compilation(), "");
+ if (task->is_unloaded()) {
+ return true; // unloaded
+ }
+ task->method()->set_queued_for_compilation(); // FIXME
+ if (task->method()->pending_queue_processed()) {
+ return true; // already queued
+ }
+ // Mark the method as being in the compile queue.
+ task->method()->set_pending_queue_processed();
+ if (CompileBroker::compilation_is_complete(task->method(), task->osr_bci(), task->comp_level(),
+ task->requires_online_compilation(), task->compile_reason())) {
+ return true; // already compiled
+ }
+ return false; // active
+ }
+
+ void CompileQueue::transfer_pending() {
+ assert(_lock->owned_by_self(), "must own lock");
+ while (!_queue.empty()) {
+ CompileTask* task = _queue.pop();
+ bool is_stale = process_pending(task);
+ if (is_stale) {
+ task->set_next(_first_stale);
+ task->set_prev(nullptr);
+ _first_stale = task;
+ } else {
+ add(task);
+ }
+ }
}
/**
* Empties compilation queue by putting all compilation tasks onto
* a freelist. Furthermore, the method wakes up all threads that are
* waiting on a compilation task to finish. This can happen if background
* compilation is disabled.
*/
void CompileQueue::free_all() {
! MutexLocker mu(_lock);
+ transfer_pending();
+
CompileTask* next = _first;
// Iterate over all tasks in the compile queue
while (next != nullptr) {
CompileTask* current = next;
}
_first = nullptr;
_last = nullptr;
// Wake up all threads that block on the queue.
! MethodCompileQueue_lock->notify_all();
}
/**
* Get the next CompileTask from a CompileQueue
*/
CompileTask* CompileQueue::get(CompilerThread* thread) {
// save methods from RedefineClasses across safepoint
! // across MethodCompileQueue_lock below.
methodHandle save_method;
methodHandle save_hot_method;
! MonitorLocker locker(MethodCompileQueue_lock);
// If _first is null we have no more compile jobs. There are two reasons for
// having no compile jobs: First, we compiled everything we wanted. Second,
// we ran out of code cache so compilation has been disabled. In the latter
// case we perform code cache sweeps to free memory such that we can re-enable
// compilation.
}
_first = nullptr;
_last = nullptr;
// Wake up all threads that block on the queue.
! _lock->notify_all();
}
/**
* Get the next CompileTask from a CompileQueue
*/
CompileTask* CompileQueue::get(CompilerThread* thread) {
// save methods from RedefineClasses across safepoint
! // across compile queue lock below.
methodHandle save_method;
methodHandle save_hot_method;
! MonitorLocker locker(_lock);
+ transfer_pending();
+
+ CompilationPolicy::sample_load_average();
+
// If _first is null we have no more compile jobs. There are two reasons for
// having no compile jobs: First, we compiled everything we wanted. Second,
// we ran out of code cache so compilation has been disabled. In the latter
// case we perform code cache sweeps to free memory such that we can re-enable
// compilation.
// We need a timed wait here, since compiler threads can exit if compilation
// is disabled forever. We use 5 seconds wait time; the exiting of compiler threads
// is not critical and we do not want idle compiler threads to wake up too often.
locker.wait(5*1000);
+ transfer_pending(); // reacquired lock
+
+ if (CompilationPolicy::have_recompilation_work()) return nullptr;
+
if (UseDynamicNumberOfCompilerThreads && _first == nullptr) {
// Still nothing to compile. Give caller a chance to stop this thread.
if (CompileBroker::can_remove(CompilerThread::current(), false)) return nullptr;
}
}
}
CompileTask* task;
{
NoSafepointVerifier nsv;
! task = CompilationPolicy::select_task(this);
if (task != nullptr) {
task = task->select_for_compilation();
}
}
}
CompileTask* task;
{
NoSafepointVerifier nsv;
! task = CompilationPolicy::select_task(this, thread);
if (task != nullptr) {
task = task->select_for_compilation();
}
}
}
// Clean & deallocate stale compile tasks.
// Temporarily releases MethodCompileQueue lock.
void CompileQueue::purge_stale_tasks() {
! assert(MethodCompileQueue_lock->owned_by_self(), "must own lock");
if (_first_stale != nullptr) {
// Stale tasks are purged when MCQ lock is released,
// but _first_stale updates are protected by MCQ lock.
// Once task processing starts and MCQ lock is released,
// other compiler threads can reuse _first_stale.
CompileTask* head = _first_stale;
_first_stale = nullptr;
{
! MutexUnlocker ul(MethodCompileQueue_lock);
for (CompileTask* task = head; task != nullptr; ) {
CompileTask* next_task = task->next();
CompileTaskWrapper ctw(task); // Frees the task
task->set_failure_reason("stale task");
task = next_task;
}
}
}
}
void CompileQueue::remove(CompileTask* task) {
! assert(MethodCompileQueue_lock->owned_by_self(), "must own lock");
if (task->prev() != nullptr) {
task->prev()->set_next(task->next());
} else {
// max is the first element
assert(task == _first, "Sanity");
}
// Clean & deallocate stale compile tasks.
// Temporarily releases MethodCompileQueue lock.
void CompileQueue::purge_stale_tasks() {
! assert(_lock->owned_by_self(), "must own lock");
if (_first_stale != nullptr) {
// Stale tasks are purged when MCQ lock is released,
// but _first_stale updates are protected by MCQ lock.
// Once task processing starts and MCQ lock is released,
// other compiler threads can reuse _first_stale.
CompileTask* head = _first_stale;
_first_stale = nullptr;
{
! MutexUnlocker ul(_lock);
for (CompileTask* task = head; task != nullptr; ) {
CompileTask* next_task = task->next();
CompileTaskWrapper ctw(task); // Frees the task
task->set_failure_reason("stale task");
task = next_task;
}
}
+ transfer_pending(); // transfer pending after reacquiring MCQ lock
}
}
void CompileQueue::remove(CompileTask* task) {
! assert(_lock->owned_by_self(), "must own lock");
if (task->prev() != nullptr) {
task->prev()->set_next(task->next());
} else {
// max is the first element
assert(task == _first, "Sanity");
--_size;
++_total_removed;
}
void CompileQueue::remove_and_mark_stale(CompileTask* task) {
! assert(MethodCompileQueue_lock->owned_by_self(), "must own lock");
remove(task);
// Enqueue the task for reclamation (should be done outside MCQ lock)
task->set_next(_first_stale);
task->set_prev(nullptr);
--_size;
++_total_removed;
}
void CompileQueue::remove_and_mark_stale(CompileTask* task) {
! assert(_lock->owned_by_self(), "must own lock");
remove(task);
// Enqueue the task for reclamation (should be done outside MCQ lock)
task->set_next(_first_stale);
task->set_prev(nullptr);
}
// methods in the compile queue need to be marked as used on the stack
// so that they don't get reclaimed by Redefine Classes
void CompileQueue::mark_on_stack() {
! CompileTask* task = _first;
! while (task != nullptr) {
task->mark_on_stack();
- task = task->next();
}
}
! CompileQueue* CompileBroker::compile_queue(int comp_level) {
! if (is_c2_compile(comp_level)) return _c2_compile_queue;
! if (is_c1_compile(comp_level)) return _c1_compile_queue;
return nullptr;
}
CompileQueue* CompileBroker::c1_compile_queue() {
return _c1_compile_queue;
}
// methods in the compile queue need to be marked as used on the stack
// so that they don't get reclaimed by Redefine Classes
void CompileQueue::mark_on_stack() {
! for (CompileTask* task = _first; task != nullptr; task = task->next()) {
! task->mark_on_stack();
+ }
+ for (CompileTask* task = _queue.first(); !_queue.is_end(task); task = task->next()) {
+ assert(task != nullptr, "");
task->mark_on_stack();
}
}
! CompileQueue* CompileBroker::compile_queue(int comp_level, bool is_scc) {
! if (is_c2_compile(comp_level)) return ((is_scc && (_sc_count > 0)) ? _sc2_compile_queue : _c2_compile_queue);
! if (is_c1_compile(comp_level)) return ((is_scc && (_sc_count > 0)) ? _sc1_compile_queue : _c1_compile_queue);
return nullptr;
}
CompileQueue* CompileBroker::c1_compile_queue() {
return _c1_compile_queue;
_c1_compile_queue->print(st);
}
if (_c2_compile_queue != nullptr) {
_c2_compile_queue->print(st);
}
}
void CompileQueue::print(outputStream* st) {
! assert_locked_or_safepoint(MethodCompileQueue_lock);
st->print_cr("%s:", name());
CompileTask* task = _first;
if (task == nullptr) {
st->print_cr("Empty");
} else {
_c1_compile_queue->print(st);
}
if (_c2_compile_queue != nullptr) {
_c2_compile_queue->print(st);
}
+ if (_c3_compile_queue != nullptr) {
+ _c3_compile_queue->print(st);
+ }
+ if (_sc1_compile_queue != nullptr) {
+ _sc1_compile_queue->print(st);
+ }
+ if (_sc2_compile_queue != nullptr) {
+ _sc2_compile_queue->print(st);
+ }
}
void CompileQueue::print(outputStream* st) {
! assert_locked_or_safepoint(_lock);
st->print_cr("%s:", name());
CompileTask* task = _first;
if (task == nullptr) {
st->print_cr("Empty");
} else {
return;
}
// Set the interface to the current compiler(s).
_c1_count = CompilationPolicy::c1_count();
_c2_count = CompilationPolicy::c2_count();
+ _c3_count = CompilationPolicy::c3_count();
+ _sc_count = CompilationPolicy::sc_count();
#if INCLUDE_JVMCI
if (EnableJVMCI) {
// This is creating a JVMCICompiler singleton.
JVMCICompiler* jvmci = new JVMCICompiler();
} else {
#ifdef COMPILER1
_c1_count = JVMCIHostThreads;
#endif // COMPILER1
}
+ #ifdef COMPILER2
+ if (SCCache::is_on() && (_c3_count > 0)) {
+ _compilers[2] = new C2Compiler();
+ }
+ #endif
}
}
#endif // INCLUDE_JVMCI
#ifdef COMPILER1
PerfData::U_None,
(jlong)CompileBroker::no_compile,
CHECK);
}
+ log_info(scc, init)("CompileBroker is initialized");
_initialized = true;
}
+ Handle CompileBroker::create_thread_oop(const char* name, TRAPS) {
+ Handle thread_oop = JavaThread::create_system_thread_object(name, CHECK_NH);
+ return thread_oop;
+ }
+
+ void TrainingReplayThread::training_replay_thread_entry(JavaThread* thread, TRAPS) {
+ CompilationPolicy::replay_training_at_init_loop(thread);
+ }
+
#if defined(ASSERT) && COMPILER2_OR_JVMCI
// Stress testing. Dedicated threads revert optimizations based on escape analysis concurrently to
// the running java application. Configured with vm options DeoptimizeObjectsALot*.
class DeoptimizeObjectsALotThread : public JavaThread {
#if defined(ASSERT) && COMPILER2_OR_JVMCI
case deoptimizer_t:
new_thread = new DeoptimizeObjectsALotThread();
break;
#endif // ASSERT
+ case training_replay_t:
+ new_thread = new TrainingReplayThread();
+ break;
default:
ShouldNotReachHere();
}
// At this point the new CompilerThread data-races with this startup
LogStream ls(lt);
ls.print_cr("%s", msg.as_string());
}
}
void CompileBroker::init_compiler_threads() {
// Ensure any exceptions lead to vm_exit_during_initialization.
EXCEPTION_MARK;
#if !defined(ZERO)
assert(_c2_count > 0 || _c1_count > 0, "No compilers?");
#endif // !ZERO
// Initialize the compilation queue
if (_c2_count > 0) {
const char* name = JVMCI_ONLY(UseJVMCICompiler ? "JVMCI compile queue" :) "C2 compile queue";
! _c2_compile_queue = new CompileQueue(name);
_compiler2_objects = NEW_C_HEAP_ARRAY(jobject, _c2_count, mtCompiler);
_compiler2_logs = NEW_C_HEAP_ARRAY(CompileLog*, _c2_count, mtCompiler);
}
if (_c1_count > 0) {
! _c1_compile_queue = new CompileQueue("C1 compile queue");
_compiler1_objects = NEW_C_HEAP_ARRAY(jobject, _c1_count, mtCompiler);
_compiler1_logs = NEW_C_HEAP_ARRAY(CompileLog*, _c1_count, mtCompiler);
}
for (int i = 0; i < _c2_count; i++) {
// Create a name for our thread.
jobject thread_handle = create_compiler_thread(_compilers[1], i, CHECK);
_compiler2_objects[i] = thread_handle;
_compiler2_logs[i] = nullptr;
if (!UseDynamicNumberOfCompilerThreads || i == 0) {
JavaThread *ct = make_thread(compiler_t, thread_handle, _c2_compile_queue, _compilers[1], THREAD);
assert(ct != nullptr, "should have been handled for initial thread");
_compilers[1]->set_num_compiler_threads(i + 1);
! if (trace_compiler_threads()) {
- ResourceMark rm;
- ThreadsListHandle tlh; // name() depends on the TLH.
- assert(tlh.includes(ct), "ct=" INTPTR_FORMAT " exited unexpectedly.", p2i(ct));
- stringStream msg;
- msg.print("Added initial compiler thread %s", ct->name());
- print_compiler_threads(msg);
- }
}
}
for (int i = 0; i < _c1_count; i++) {
// Create a name for our thread.
LogStream ls(lt);
ls.print_cr("%s", msg.as_string());
}
}
+ static void print_compiler_thread(JavaThread *ct) {
+ if (trace_compiler_threads()) {
+ ResourceMark rm;
+ ThreadsListHandle tlh; // name() depends on the TLH.
+ assert(tlh.includes(ct), "ct=" INTPTR_FORMAT " exited unexpectedly.", p2i(ct));
+ stringStream msg;
+ msg.print("Added initial compiler thread %s", ct->name());
+ print_compiler_threads(msg);
+ }
+ }
+
void CompileBroker::init_compiler_threads() {
// Ensure any exceptions lead to vm_exit_during_initialization.
EXCEPTION_MARK;
#if !defined(ZERO)
assert(_c2_count > 0 || _c1_count > 0, "No compilers?");
#endif // !ZERO
// Initialize the compilation queue
if (_c2_count > 0) {
const char* name = JVMCI_ONLY(UseJVMCICompiler ? "JVMCI compile queue" :) "C2 compile queue";
! _c2_compile_queue = new CompileQueue(name, MethodCompileQueueC2_lock);
_compiler2_objects = NEW_C_HEAP_ARRAY(jobject, _c2_count, mtCompiler);
_compiler2_logs = NEW_C_HEAP_ARRAY(CompileLog*, _c2_count, mtCompiler);
}
if (_c1_count > 0) {
! _c1_compile_queue = new CompileQueue("C1 compile queue", MethodCompileQueueC1_lock);
_compiler1_objects = NEW_C_HEAP_ARRAY(jobject, _c1_count, mtCompiler);
_compiler1_logs = NEW_C_HEAP_ARRAY(CompileLog*, _c1_count, mtCompiler);
}
+ if (_c3_count > 0) {
+ const char* name = "C2 compile queue";
+ _c3_compile_queue = new CompileQueue(name, MethodCompileQueueC3_lock);
+ _compiler3_objects = NEW_C_HEAP_ARRAY(jobject, _c3_count, mtCompiler);
+ _compiler3_logs = NEW_C_HEAP_ARRAY(CompileLog*, _c3_count, mtCompiler);
+ }
+ if (_sc_count > 0) {
+ if (_c1_count > 0) { // C1 is present
+ _sc1_compile_queue = new CompileQueue("C1 SC compile queue", MethodCompileQueueSC1_lock);
+ }
+ if (_c2_count > 0) { // C2 is present
+ _sc2_compile_queue = new CompileQueue("C2 SC compile queue", MethodCompileQueueSC2_lock);
+ }
+ _sc_objects = NEW_C_HEAP_ARRAY(jobject, _sc_count, mtCompiler);
+ _sc_logs = NEW_C_HEAP_ARRAY(CompileLog*, _sc_count, mtCompiler);
+ }
+ char name_buffer[256];
+
for (int i = 0; i < _c2_count; i++) {
// Create a name for our thread.
jobject thread_handle = create_compiler_thread(_compilers[1], i, CHECK);
_compiler2_objects[i] = thread_handle;
_compiler2_logs[i] = nullptr;
if (!UseDynamicNumberOfCompilerThreads || i == 0) {
JavaThread *ct = make_thread(compiler_t, thread_handle, _c2_compile_queue, _compilers[1], THREAD);
assert(ct != nullptr, "should have been handled for initial thread");
_compilers[1]->set_num_compiler_threads(i + 1);
! print_compiler_thread(ct);
}
}
for (int i = 0; i < _c1_count; i++) {
// Create a name for our thread.
if (!UseDynamicNumberOfCompilerThreads || i == 0) {
JavaThread *ct = make_thread(compiler_t, thread_handle, _c1_compile_queue, _compilers[0], THREAD);
assert(ct != nullptr, "should have been handled for initial thread");
_compilers[0]->set_num_compiler_threads(i + 1);
! if (trace_compiler_threads()) {
! ResourceMark rm;
! ThreadsListHandle tlh; // name() depends on the TLH.
! assert(tlh.includes(ct), "ct=" INTPTR_FORMAT " exited unexpectedly.", p2i(ct));
! stringStream msg;
! msg.print("Added initial compiler thread %s", ct->name());
! print_compiler_threads(msg);
! }
}
}
if (UsePerfData) {
! PerfDataManager::create_constant(SUN_CI, "threads", PerfData::U_Bytes, _c1_count + _c2_count, CHECK);
}
#if defined(ASSERT) && COMPILER2_OR_JVMCI
if (DeoptimizeObjectsALot) {
// Initialize and start the object deoptimizer threads
if (!UseDynamicNumberOfCompilerThreads || i == 0) {
JavaThread *ct = make_thread(compiler_t, thread_handle, _c1_compile_queue, _compilers[0], THREAD);
assert(ct != nullptr, "should have been handled for initial thread");
_compilers[0]->set_num_compiler_threads(i + 1);
! print_compiler_thread(ct);
! }
! }
!
! for (int i = 0; i < _c3_count; i++) {
! // Create a name for our thread.
! os::snprintf_checked(name_buffer, sizeof(name_buffer), "C2 CompilerThread%d", i);
! Handle thread_oop = create_thread_oop(name_buffer, CHECK);
+ jobject thread_handle = JNIHandles::make_global(thread_oop);
+ _compiler3_objects[i] = thread_handle;
+ _compiler3_logs[i] = nullptr;
+
+ JavaThread *ct = make_thread(compiler_t, thread_handle, _c3_compile_queue, _compilers[2], THREAD);
+ assert(ct != nullptr, "should have been handled for initial thread");
+ _compilers[2]->set_num_compiler_threads(i + 1);
+ print_compiler_thread(ct);
+ }
+
+ if (_sc_count > 0) {
+ int i = 0;
+ if (_c1_count > 0) { // C1 is present
+ os::snprintf_checked(name_buffer, sizeof(name_buffer), "C%d SC CompilerThread", 1);
+ Handle thread_oop = create_thread_oop(name_buffer, CHECK);
+ jobject thread_handle = JNIHandles::make_global(thread_oop);
+ _sc_objects[i] = thread_handle;
+ _sc_logs[i] = nullptr;
+ i++;
+
+ JavaThread *ct = make_thread(compiler_t, thread_handle, _sc1_compile_queue, _compilers[0], THREAD);
+ assert(ct != nullptr, "should have been handled for initial thread");
+ print_compiler_thread(ct);
+ }
+ if (_c2_count > 0) { // C2 is present
+ os::snprintf_checked(name_buffer, sizeof(name_buffer), "C%d SC CompilerThread", 2);
+ Handle thread_oop = create_thread_oop(name_buffer, CHECK);
+ jobject thread_handle = JNIHandles::make_global(thread_oop);
+ _sc_objects[i] = thread_handle;
+ _sc_logs[i] = nullptr;
+
+ JavaThread *ct = make_thread(compiler_t, thread_handle, _sc2_compile_queue, _compilers[1], THREAD);
+ assert(ct != nullptr, "should have been handled for initial thread");
+ print_compiler_thread(ct);
}
}
if (UsePerfData) {
! PerfDataManager::create_constant(SUN_CI, "threads", PerfData::U_Bytes, _c1_count + _c2_count + _c3_count, CHECK);
}
#if defined(ASSERT) && COMPILER2_OR_JVMCI
if (DeoptimizeObjectsALot) {
// Initialize and start the object deoptimizer threads
}
}
#endif // defined(ASSERT) && COMPILER2_OR_JVMCI
}
+ void CompileBroker::init_training_replay() {
+ // Ensure any exceptions lead to vm_exit_during_initialization.
+ EXCEPTION_MARK;
+ if (TrainingData::have_data()) {
+ if (UseConcurrentTrainingReplay) {
+ Handle thread_oop = create_thread_oop("Training replay thread", CHECK);
+ jobject thread_handle = JNIHandles::make_local(THREAD, thread_oop());
+ make_thread(training_replay_t, thread_handle, nullptr, nullptr, THREAD);
+ }
+ _replay_initialized = true;
+ }
+ }
+
void CompileBroker::possibly_add_compiler_threads(JavaThread* THREAD) {
julong free_memory = os::free_memory();
// If SegmentedCodeCache is off, both values refer to the single heap (with type CodeBlobType::All).
size_t available_cc_np = CodeCache::unallocated_capacity(CodeBlobType::MethodNonProfiled),
*/
void CompileBroker::mark_on_stack() {
assert(SafepointSynchronize::is_at_safepoint(), "sanity check");
// Since we are at a safepoint, we do not need a lock to access
// the compile queues.
+ if (_c3_compile_queue != nullptr) {
+ _c3_compile_queue->mark_on_stack();
+ }
if (_c2_compile_queue != nullptr) {
_c2_compile_queue->mark_on_stack();
}
if (_c1_compile_queue != nullptr) {
_c1_compile_queue->mark_on_stack();
}
+ if (_sc1_compile_queue != nullptr) {
+ _sc1_compile_queue->mark_on_stack();
+ }
+ if (_sc2_compile_queue != nullptr) {
+ _sc2_compile_queue->mark_on_stack();
+ }
}
// ------------------------------------------------------------------
// CompileBroker::compile_method
//
int osr_bci,
int comp_level,
const methodHandle& hot_method,
int hot_count,
CompileTask::CompileReason compile_reason,
bool blocking,
Thread* thread) {
guarantee(!method->is_abstract(), "cannot compile abstract methods");
assert(method->method_holder()->is_instance_klass(),
"sanity check");
! assert(!method->method_holder()->is_not_initialized(),
! "method holder must be initialized");
assert(!method->is_method_handle_intrinsic(), "do not enqueue these guys");
if (CIPrintRequests) {
tty->print("request: ");
method->print_short_name(tty);
int osr_bci,
int comp_level,
const methodHandle& hot_method,
int hot_count,
CompileTask::CompileReason compile_reason,
+ bool requires_online_compilation,
bool blocking,
Thread* thread) {
guarantee(!method->is_abstract(), "cannot compile abstract methods");
assert(method->method_holder()->is_instance_klass(),
"sanity check");
! assert(!method->method_holder()->is_not_initialized() ||
! compile_reason == CompileTask::Reason_Preload ||
+ compile_reason == CompileTask::Reason_Precompile ||
+ compile_reason == CompileTask::Reason_PrecompileForPreload, "method holder must be initialized");
assert(!method->is_method_handle_intrinsic(), "do not enqueue these guys");
if (CIPrintRequests) {
tty->print("request: ");
method->print_short_name(tty);
}
// A request has been made for compilation. Before we do any
// real work, check to see if the method has been compiled
// in the meantime with a definitive result.
! if (compilation_is_complete(method, osr_bci, comp_level)) {
return;
}
#ifndef PRODUCT
if (osr_bci != -1 && !FLAG_IS_DEFAULT(OSROnlyBCI)) {
}
// A request has been made for compilation. Before we do any
// real work, check to see if the method has been compiled
// in the meantime with a definitive result.
! if (compilation_is_complete(method(), osr_bci, comp_level, requires_online_compilation, compile_reason)) {
return;
}
#ifndef PRODUCT
if (osr_bci != -1 && !FLAG_IS_DEFAULT(OSROnlyBCI)) {
return;
}
// Tiered policy requires MethodCounters to exist before adding a method to
// the queue. Create if we don't have them yet.
! method->get_method_counters(thread);
// Outputs from the following MutexLocker block:
! CompileTask* task = nullptr;
! CompileQueue* queue = compile_queue(comp_level);
// Acquire our lock.
{
! MutexLocker locker(thread, MethodCompileQueue_lock);
// Make sure the method has not slipped into the queues since
// last we checked; note that those checks were "fast bail-outs".
// Here we need to be more careful, see 14012000 below.
if (compilation_is_in_queue(method)) {
return;
}
// Tiered policy requires MethodCounters to exist before adding a method to
// the queue. Create if we don't have them yet.
! if (compile_reason != CompileTask::Reason_Preload) {
+ method->get_method_counters(thread);
+ }
+
+ SCCEntry* scc_entry = find_scc_entry(method, osr_bci, comp_level, compile_reason, requires_online_compilation);
+ bool is_scc = (scc_entry != nullptr);
// Outputs from the following MutexLocker block:
! CompileTask* task = nullptr;
! CompileQueue* queue;
+ #if INCLUDE_JVMCI
+ if (is_c2_compile(comp_level) && compiler2()->is_jvmci() && compiler3() != nullptr &&
+ ((JVMCICompiler*)compiler2())->force_comp_at_level_simple(method)) {
+ assert(_c3_compile_queue != nullptr, "sanity");
+ queue = _c3_compile_queue; // JVMCI compiler's methods compilation
+ } else
+ #endif
+ queue = compile_queue(comp_level, is_scc);
// Acquire our lock.
{
! ConditionalMutexLocker locker(thread, queue->lock(), !UseLockFreeCompileQueues);
// Make sure the method has not slipped into the queues since
// last we checked; note that those checks were "fast bail-outs".
// Here we need to be more careful, see 14012000 below.
if (compilation_is_in_queue(method)) {
}
// We need to check again to see if the compilation has
// completed. A previous compilation may have registered
// some result.
! if (compilation_is_complete(method, osr_bci, comp_level)) {
return;
}
// We now know that this compilation is not pending, complete,
// or prohibited. Assign a compile_id to this compilation
}
// We need to check again to see if the compilation has
// completed. A previous compilation may have registered
// some result.
! if (compilation_is_complete(method(), osr_bci, comp_level, requires_online_compilation, compile_reason)) {
return;
}
// We now know that this compilation is not pending, complete,
// or prohibited. Assign a compile_id to this compilation
// these bits, and their updating (here and elsewhere) under a
// common lock.
task = create_compile_task(queue,
compile_id, method,
osr_bci, comp_level,
! hot_method, hot_count, compile_reason,
! blocking);
}
if (blocking) {
wait_for_completion(task);
}
}
nmethod* CompileBroker::compile_method(const methodHandle& method, int osr_bci,
int comp_level,
const methodHandle& hot_method, int hot_count,
CompileTask::CompileReason compile_reason,
TRAPS) {
// Do nothing if compilebroker is not initialized or compiles are submitted on level none
if (!_initialized || comp_level == CompLevel_none) {
return nullptr;
}
AbstractCompiler *comp = CompileBroker::compiler(comp_level);
assert(comp != nullptr, "Ensure we have a compiler");
#if INCLUDE_JVMCI
if (comp->is_jvmci() && !JVMCI::can_initialize_JVMCI()) {
// these bits, and their updating (here and elsewhere) under a
// common lock.
task = create_compile_task(queue,
compile_id, method,
osr_bci, comp_level,
! hot_method, hot_count, scc_entry, compile_reason,
! requires_online_compilation, blocking);
+
+ if (task->is_scc() && (_sc_count > 0)) {
+ // Put it on SC queue
+ queue = is_c1_compile(comp_level) ? _sc1_compile_queue : _sc2_compile_queue;
+ }
+
+ if (UseLockFreeCompileQueues) {
+ assert(queue->lock()->owned_by_self() == false, "");
+ queue->add_pending(task);
+ } else {
+ queue->add(task);
+ }
}
if (blocking) {
wait_for_completion(task);
}
}
+ SCCEntry* CompileBroker::find_scc_entry(const methodHandle& method, int osr_bci, int comp_level,
+ CompileTask::CompileReason compile_reason,
+ bool requires_online_compilation) {
+ SCCEntry* scc_entry = nullptr;
+ if (osr_bci == InvocationEntryBci && !requires_online_compilation && SCCache::is_on_for_read()) {
+ // Check for cached code.
+ if (compile_reason == CompileTask::Reason_Preload) {
+ scc_entry = method->scc_entry();
+ assert(scc_entry != nullptr && scc_entry->for_preload(), "sanity");
+ } else {
+ scc_entry = SCCache::find_code_entry(method, comp_level);
+ }
+ }
+ return scc_entry;
+ }
+
nmethod* CompileBroker::compile_method(const methodHandle& method, int osr_bci,
int comp_level,
const methodHandle& hot_method, int hot_count,
+ bool requires_online_compilation,
CompileTask::CompileReason compile_reason,
TRAPS) {
// Do nothing if compilebroker is not initialized or compiles are submitted on level none
if (!_initialized || comp_level == CompLevel_none) {
return nullptr;
}
+ #if INCLUDE_JVMCI
+ if (EnableJVMCI && UseJVMCICompiler &&
+ comp_level == CompLevel_full_optimization && !AOTLinkedClassBulkLoader::class_preloading_finished()) {
+ return nullptr;
+ }
+ #endif
+
AbstractCompiler *comp = CompileBroker::compiler(comp_level);
assert(comp != nullptr, "Ensure we have a compiler");
#if INCLUDE_JVMCI
if (comp->is_jvmci() && !JVMCI::can_initialize_JVMCI()) {
}
#endif
DirectiveSet* directive = DirectivesStack::getMatchingDirective(method, comp);
// CompileBroker::compile_method can trap and can have pending async exception.
! nmethod* nm = CompileBroker::compile_method(method, osr_bci, comp_level, hot_method, hot_count, compile_reason, directive, THREAD);
DirectivesStack::release(directive);
return nm;
}
nmethod* CompileBroker::compile_method(const methodHandle& method, int osr_bci,
int comp_level,
const methodHandle& hot_method, int hot_count,
CompileTask::CompileReason compile_reason,
DirectiveSet* directive,
TRAPS) {
// make sure arguments make sense
assert(method->method_holder()->is_instance_klass(), "not an instance method");
assert(osr_bci == InvocationEntryBci || (0 <= osr_bci && osr_bci < method->code_size()), "bci out of range");
assert(!method->is_abstract() && (osr_bci == InvocationEntryBci || !method->is_native()), "cannot compile abstract/native methods");
! assert(!method->method_holder()->is_not_initialized(), "method holder must be initialized");
// return quickly if possible
// lock, make sure that the compilation
// isn't prohibited in a straightforward way.
AbstractCompiler* comp = CompileBroker::compiler(comp_level);
if (comp == nullptr || compilation_is_prohibited(method, osr_bci, comp_level, directive->ExcludeOption)) {
return nullptr;
}
#endif
DirectiveSet* directive = DirectivesStack::getMatchingDirective(method, comp);
// CompileBroker::compile_method can trap and can have pending async exception.
! nmethod* nm = CompileBroker::compile_method(method, osr_bci, comp_level, hot_method, hot_count, requires_online_compilation, compile_reason, directive, THREAD);
DirectivesStack::release(directive);
return nm;
}
nmethod* CompileBroker::compile_method(const methodHandle& method, int osr_bci,
int comp_level,
const methodHandle& hot_method, int hot_count,
+ bool requires_online_compilation,
CompileTask::CompileReason compile_reason,
DirectiveSet* directive,
TRAPS) {
// make sure arguments make sense
assert(method->method_holder()->is_instance_klass(), "not an instance method");
assert(osr_bci == InvocationEntryBci || (0 <= osr_bci && osr_bci < method->code_size()), "bci out of range");
assert(!method->is_abstract() && (osr_bci == InvocationEntryBci || !method->is_native()), "cannot compile abstract/native methods");
! assert(!method->method_holder()->is_not_initialized() ||
+ compile_reason == CompileTask::Reason_Preload ||
+ compile_reason == CompileTask::Reason_Precompile ||
+ compile_reason == CompileTask::Reason_PrecompileForPreload, "method holder must be initialized");
// return quickly if possible
+ if (PrecompileOnlyAndExit && !CompileTask::reason_is_precompiled(compile_reason)) {
+ return nullptr;
+ }
+
// lock, make sure that the compilation
// isn't prohibited in a straightforward way.
AbstractCompiler* comp = CompileBroker::compiler(comp_level);
if (comp == nullptr || compilation_is_prohibited(method, osr_bci, comp_level, directive->ExcludeOption)) {
return nullptr;
if (osr_bci == InvocationEntryBci) {
// standard compilation
nmethod* method_code = method->code();
if (method_code != nullptr) {
! if (compilation_is_complete(method, osr_bci, comp_level)) {
return method_code;
}
}
if (method->is_not_compilable(comp_level)) {
return nullptr;
if (osr_bci == InvocationEntryBci) {
// standard compilation
nmethod* method_code = method->code();
if (method_code != nullptr) {
! if (compilation_is_complete(method(), osr_bci, comp_level, requires_online_compilation, compile_reason)) {
return method_code;
}
}
if (method->is_not_compilable(comp_level)) {
return nullptr;
if (method->is_not_osr_compilable(comp_level)) return nullptr;
}
assert(!HAS_PENDING_EXCEPTION, "No exception should be present");
// some prerequisites that are compiler specific
! if (comp->is_c2() || comp->is_jvmci()) {
InternalOOMEMark iom(THREAD);
method->constants()->resolve_string_constants(CHECK_AND_CLEAR_NONASYNC_NULL);
// Resolve all classes seen in the signature of the method
// we are compiling.
Method::load_signature_classes(method, CHECK_AND_CLEAR_NONASYNC_NULL);
if (method->is_not_osr_compilable(comp_level)) return nullptr;
}
assert(!HAS_PENDING_EXCEPTION, "No exception should be present");
// some prerequisites that are compiler specific
! if (compile_reason != CompileTask::Reason_Preload && (comp->is_c2() || comp->is_jvmci())) {
InternalOOMEMark iom(THREAD);
method->constants()->resolve_string_constants(CHECK_AND_CLEAR_NONASYNC_NULL);
// Resolve all classes seen in the signature of the method
// we are compiling.
Method::load_signature_classes(method, CHECK_AND_CLEAR_NONASYNC_NULL);
// If the compiler is shut off due to code cache getting full
// fail out now so blocking compiles dont hang the java thread
if (!should_compile_new_jobs()) {
return nullptr;
}
! bool is_blocking = !directive->BackgroundCompilationOption || ReplayCompiles;
! compile_method_base(method, osr_bci, comp_level, hot_method, hot_count, compile_reason, is_blocking, THREAD);
}
// return requested nmethod
// We accept a higher level osr method
if (osr_bci == InvocationEntryBci) {
// If the compiler is shut off due to code cache getting full
// fail out now so blocking compiles dont hang the java thread
if (!should_compile_new_jobs()) {
return nullptr;
}
! bool is_blocking = ReplayCompiles ||
! !directive->BackgroundCompilationOption ||
+ (compile_reason == CompileTask::Reason_Precompile) ||
+ (compile_reason == CompileTask::Reason_PrecompileForPreload);
+ compile_method_base(method, osr_bci, comp_level, hot_method, hot_count, compile_reason, requires_online_compilation, is_blocking, THREAD);
}
// return requested nmethod
// We accept a higher level osr method
if (osr_bci == InvocationEntryBci) {
// ------------------------------------------------------------------
// CompileBroker::compilation_is_complete
//
// See if compilation of this method is already complete.
! bool CompileBroker::compilation_is_complete(const methodHandle& method,
! int osr_bci,
! int comp_level) {
bool is_osr = (osr_bci != standard_entry_bci);
if (is_osr) {
if (method->is_not_osr_compilable(comp_level)) {
return true;
} else {
// ------------------------------------------------------------------
// CompileBroker::compilation_is_complete
//
// See if compilation of this method is already complete.
! bool CompileBroker::compilation_is_complete(Method* method,
! int osr_bci,
! int comp_level,
+ bool online_only,
+ CompileTask::CompileReason compile_reason) {
+ if (compile_reason == CompileTask::Reason_Precompile ||
+ compile_reason == CompileTask::Reason_PrecompileForPreload) {
+ return false; // FIXME: any restrictions?
+ }
bool is_osr = (osr_bci != standard_entry_bci);
if (is_osr) {
if (method->is_not_osr_compilable(comp_level)) {
return true;
} else {
} else {
if (method->is_not_compilable(comp_level)) {
return true;
} else {
nmethod* result = method->code();
! if (result == nullptr) return false;
! return comp_level == result->comp_level();
}
}
}
} else {
if (method->is_not_compilable(comp_level)) {
return true;
} else {
nmethod* result = method->code();
! if (result == nullptr) {
! return false;
+ }
+ if (online_only && result->is_scc()) {
+ return false;
+ }
+ bool same_level = (comp_level == result->comp_level());
+ if (result->has_clinit_barriers()) {
+ return !same_level; // Allow replace preloaded code with new code of the same level
+ }
+ return same_level;
}
}
}
// ------------------------------------------------------------------
// CompileBroker::assign_compile_id_unlocked
//
// Public wrapper for assign_compile_id that acquires the needed locks
int CompileBroker::assign_compile_id_unlocked(Thread* thread, const methodHandle& method, int osr_bci) {
- MutexLocker locker(thread, MethodCompileQueue_lock);
return assign_compile_id(method, osr_bci);
}
// ------------------------------------------------------------------
// CompileBroker::create_compile_task
const methodHandle& method,
int osr_bci,
int comp_level,
const methodHandle& hot_method,
int hot_count,
CompileTask::CompileReason compile_reason,
bool blocking) {
CompileTask* new_task = CompileTask::allocate();
new_task->initialize(compile_id, method, osr_bci, comp_level,
! hot_method, hot_count, compile_reason,
! blocking);
- queue->add(new_task);
return new_task;
}
#if INCLUDE_JVMCI
// The number of milliseconds to wait before checking if
const methodHandle& method,
int osr_bci,
int comp_level,
const methodHandle& hot_method,
int hot_count,
+ SCCEntry* scc_entry,
CompileTask::CompileReason compile_reason,
+ bool requires_online_compilation,
bool blocking) {
CompileTask* new_task = CompileTask::allocate();
new_task->initialize(compile_id, method, osr_bci, comp_level,
! hot_method, hot_count, scc_entry, compile_reason, queue,
! requires_online_compilation, blocking);
return new_task;
}
#if INCLUDE_JVMCI
// The number of milliseconds to wait before checking if
// Switch back to VM state to do compiler initialization
ThreadInVMfromNative tv(thread);
// Perform per-thread and global initializations
+ {
+ MutexLocker only_one (thread, CompileThread_lock);
+ SCCache::init_table();
+ }
comp->initialize();
}
if (comp->is_failed()) {
disable_compilation_forever();
* out to be a problem.
*/
void CompileBroker::shutdown_compiler_runtime(AbstractCompiler* comp, CompilerThread* thread) {
free_buffer_blob_if_allocated(thread);
+ log_info(compilation)("shutdown_compiler_runtime: " INTPTR_FORMAT, p2i(thread));
+
if (comp->should_perform_shutdown()) {
// There are two reasons for shutting down the compiler
// 1) compiler runtime initialization failed
// 2) The code cache is full and the following flag is set: -XX:-UseCodeCacheFlushing
warning("%s initialization failed. Shutting down all compilers", comp->name());
if (_c2_compile_queue != nullptr) {
_c2_compile_queue->free_all();
}
+ if (_c3_compile_queue != nullptr) {
+ _c3_compile_queue->free_all();
+ }
+
// Set flags so that we continue execution with using interpreter only.
UseCompiler = false;
UseInterpreter = true;
// We could delete compiler runtimes also. However, there are references to
*/
CompileLog* CompileBroker::get_log(CompilerThread* ct) {
if (!LogCompilation) return nullptr;
AbstractCompiler *compiler = ct->compiler();
bool c1 = compiler->is_c1();
! jobject* compiler_objects = c1 ? _compiler1_objects : _compiler2_objects;
assert(compiler_objects != nullptr, "must be initialized at this point");
! CompileLog** logs = c1 ? _compiler1_logs : _compiler2_logs;
assert(logs != nullptr, "must be initialized at this point");
! int count = c1 ? _c1_count : _c2_count;
// Find Compiler number by its threadObj.
oop compiler_obj = ct->threadObj();
int compiler_number = 0;
bool found = false;
for (; compiler_number < count; compiler_number++) {
*/
CompileLog* CompileBroker::get_log(CompilerThread* ct) {
if (!LogCompilation) return nullptr;
AbstractCompiler *compiler = ct->compiler();
+ bool jvmci = JVMCI_ONLY( compiler->is_jvmci() ||) false;
bool c1 = compiler->is_c1();
! jobject* compiler_objects = c1 ? _compiler1_objects : (_c3_count == 0 ? _compiler2_objects : (jvmci ? _compiler2_objects : _compiler3_objects));
assert(compiler_objects != nullptr, "must be initialized at this point");
! CompileLog** logs = c1 ? _compiler1_logs : (_c3_count == 0 ? _compiler2_logs : (jvmci ? _compiler2_logs : _compiler3_logs));
assert(logs != nullptr, "must be initialized at this point");
! int count = c1 ? _c1_count : (_c3_count == 0 ? _c2_count : (jvmci ? _c2_count : _c3_count));
+ if (ct->queue() == _sc1_compile_queue || ct->queue() == _sc2_compile_queue) {
+ compiler_objects = _sc_objects;
+ logs = _sc_logs;
+ count = _sc_count;
+ }
// Find Compiler number by its threadObj.
oop compiler_obj = ct->threadObj();
int compiler_number = 0;
bool found = false;
for (; compiler_number < count; compiler_number++) {
// space in the code cache to generate the necessary stubs, etc.
while (!is_compilation_disabled_forever()) {
// We need this HandleMark to avoid leaking VM handles.
HandleMark hm(thread);
+ CompilationPolicy::recompilation_step(RecompilationWorkUnitSize, thread);
+
CompileTask* task = queue->get(thread);
+
if (task == nullptr) {
if (UseDynamicNumberOfCompilerThreads) {
// Access compiler_count under lock to enforce consistency.
MutexLocker only_one(CompileThread_lock);
if (can_remove(thread, true)) {
invoke_compiler_on_method(task);
thread->start_idle_timer();
} else {
// After compilation is disabled, remove remaining methods from queue
method->clear_queued_for_compilation();
+ method->set_pending_queue_processed(false);
task->set_failure_reason("compilation is disabled");
}
} else {
task->set_failure_reason("breakpoints are present");
}
bool is_osr = (osr_bci != standard_entry_bci);
bool should_log = (thread->log() != nullptr);
bool should_break = false;
const int task_level = task->comp_level();
AbstractCompiler* comp = task->compiler();
+ CompileTrainingData* tdata = task->training_data();
+ assert(tdata == nullptr || TrainingData::need_data() ||
+ CDSConfig::is_dumping_preimage_static_archive(), ""); // FIXME: MetaspaceShared::preload_and_dump() messes with RecordTraining flag
{
// create the handle inside it's own block so it can't
// accidentally be referenced once the thread transitions to
// native. The NoHandleMark before the transition should catch
// any cases where this occurs in the future.
}
DTRACE_METHOD_COMPILE_BEGIN_PROBE(method, compiler_name(task_level));
}
+ if (tdata != nullptr) {
+ tdata->record_compilation_start(task);
+ }
+
should_break = directive->BreakAtCompileOption || task->check_break_at_flags();
if (should_log && !directive->LogOption) {
should_log = false;
}
ciMethod* target = ci_env.get_method_from_handle(target_handle);
TraceTime t1("compilation", &time);
EventCompilation event;
if (comp == nullptr) {
ci_env.record_method_not_compilable("no compiler");
} else if (!ci_env.failing()) {
if (WhiteBoxAPI && WhiteBox::compilation_locked) {
whitebox_lock_compilation();
}
! comp->compile_method(&ci_env, target, osr_bci, true, directive);
/* Repeat compilation without installing code for profiling purposes */
int repeat_compilation_count = directive->RepeatCompilationOption;
while (repeat_compilation_count > 0) {
ResourceMark rm(thread);
ciMethod* target = ci_env.get_method_from_handle(target_handle);
TraceTime t1("compilation", &time);
EventCompilation event;
+ bool install_code = true;
if (comp == nullptr) {
ci_env.record_method_not_compilable("no compiler");
} else if (!ci_env.failing()) {
if (WhiteBoxAPI && WhiteBox::compilation_locked) {
whitebox_lock_compilation();
}
! if (StoreCachedCode && task->is_precompiled()) {
+ install_code = false; // not suitable in the current context
+ }
+ comp->compile_method(&ci_env, target, osr_bci, install_code, directive);
/* Repeat compilation without installing code for profiling purposes */
int repeat_compilation_count = directive->RepeatCompilationOption;
while (repeat_compilation_count > 0) {
ResourceMark rm(thread);
}
}
DirectivesStack::release(directive);
! if (!ci_env.failing() && !task->is_success()) {
assert(ci_env.failure_reason() != nullptr, "expect failure reason");
assert(false, "compiler should always document failure: %s", ci_env.failure_reason());
// The compiler elected, without comment, not to register a result.
// Do not attempt further compilations of this method.
ci_env.record_method_not_compilable("compile failed");
}
}
DirectivesStack::release(directive);
! if (!ci_env.failing() && !task->is_success() && install_code) {
assert(ci_env.failure_reason() != nullptr, "expect failure reason");
assert(false, "compiler should always document failure: %s", ci_env.failure_reason());
// The compiler elected, without comment, not to register a result.
// Do not attempt further compilations of this method.
ci_env.record_method_not_compilable("compile failed");
FormatBufferResource("COMPILE SKIPPED: %s", failure_reason);
task->print(tty, msg);
}
}
methodHandle method(thread, task->method());
DTRACE_METHOD_COMPILE_END_PROBE(method, compiler_name(task_level), task->is_success());
collect_statistics(thread, time, task);
if (PrintCompilation && PrintCompilation2) {
tty->print("%7d ", (int) tty->time_stamp().milliseconds()); // print timestamp
tty->print("%4d ", compile_id); // print compilation number
! tty->print("%s ", (is_osr ? "%" : " "));
if (task->is_success()) {
tty->print("size: %d(%d) ", task->nm_total_size(), task->nm_insts_size());
}
tty->print_cr("time: %d inlined: %d bytes", (int)time.milliseconds(), task->num_inlined_bytecodes());
}
FormatBufferResource("COMPILE SKIPPED: %s", failure_reason);
task->print(tty, msg);
}
}
+ task->mark_finished(os::elapsed_counter());
+
+ if (tdata != nullptr) {
+ tdata->record_compilation_end(task);
+ }
+
methodHandle method(thread, task->method());
DTRACE_METHOD_COMPILE_END_PROBE(method, compiler_name(task_level), task->is_success());
collect_statistics(thread, time, task);
if (PrintCompilation && PrintCompilation2) {
tty->print("%7d ", (int) tty->time_stamp().milliseconds()); // print timestamp
tty->print("%4d ", compile_id); // print compilation number
! tty->print("%s ", (is_osr ? "%" : (task->is_scc() ? "A" : " ")));
if (task->is_success()) {
tty->print("size: %d(%d) ", task->nm_total_size(), task->nm_insts_size());
}
tty->print_cr("time: %d inlined: %d bytes", (int)time.milliseconds(), task->num_inlined_bytecodes());
}
// compile queue lock was held. Subsequently, we acquired the compile
// queue lock to get this task off the compile queue; thus (to belabour
// the point somewhat) our clearing of the bits must be occurring
// only after the setting of the bits. See also 14012000 above.
method->clear_queued_for_compilation();
+ method->set_pending_queue_processed(false);
+
+ if (PrintCompilation) {
+ ResourceMark rm;
+ task->print_tty();
+ }
}
/**
* The CodeCache is full. Print warning and disable compilation.
* Schedule code cache cleaning so compilation can continue later.
// updated regardless of the setting of the CITime and CITimeEach flags
//
// account all time, including bailouts and failures in this counter;
// C1 and C2 counters are counting both successful and unsuccessful compiles
! _t_total_compilation.add(time);
if (!success) {
_total_bailout_count++;
if (UsePerfData) {
_perf_last_failed_method->set_value(counters->current_method());
_perf_last_failed_type->set_value(counters->compile_type());
_perf_total_bailout_count->inc();
}
! _t_bailedout_compilation.add(time);
} else if (!task->is_success()) {
if (UsePerfData) {
_perf_last_invalidated_method->set_value(counters->current_method());
_perf_last_invalidated_type->set_value(counters->compile_type());
_perf_total_invalidated_count->inc();
}
_total_invalidated_count++;
! _t_invalidated_compilation.add(time);
} else {
// Compilation succeeded
// update compilation ticks - used by the implementation of
// java.lang.management.CompilationMXBean
_perf_total_compilation->inc(time.ticks());
_peak_compilation_time = time.milliseconds() > _peak_compilation_time ? time.milliseconds() : _peak_compilation_time;
! if (CITime) {
int bytes_compiled = method->code_size() + task->num_inlined_bytecodes();
if (is_osr) {
! _t_osr_compilation.add(time);
_sum_osr_bytes_compiled += bytes_compiled;
} else {
! _t_standard_compilation.add(time);
_sum_standard_bytes_compiled += method->code_size() + task->num_inlined_bytecodes();
}
// Collect statistic per compilation level
! if (comp_level > CompLevel_none && comp_level <= CompLevel_full_optimization) {
CompilerStatistics* stats = &_stats_per_level[comp_level-1];
if (is_osr) {
stats->_osr.update(time, bytes_compiled);
} else {
stats->_standard.update(time, bytes_compiled);
// updated regardless of the setting of the CITime and CITimeEach flags
//
// account all time, including bailouts and failures in this counter;
// C1 and C2 counters are counting both successful and unsuccessful compiles
! _t_total_compilation.add(&time);
if (!success) {
_total_bailout_count++;
if (UsePerfData) {
_perf_last_failed_method->set_value(counters->current_method());
_perf_last_failed_type->set_value(counters->compile_type());
_perf_total_bailout_count->inc();
}
! _t_bailedout_compilation.add(&time);
+
+ if (CITime || log_is_enabled(Info, init)) {
+ CompilerStatistics* stats = nullptr;
+ if (task->is_scc()) {
+ int level = task->preload() ? CompLevel_full_optimization : (comp_level - 1);
+ stats = &_scc_stats_per_level[level];
+ } else {
+ stats = &_stats_per_level[comp_level-1];
+ }
+ stats->_bailout.update(time, 0);
+ }
} else if (!task->is_success()) {
if (UsePerfData) {
_perf_last_invalidated_method->set_value(counters->current_method());
_perf_last_invalidated_type->set_value(counters->compile_type());
_perf_total_invalidated_count->inc();
}
_total_invalidated_count++;
! _t_invalidated_compilation.add(&time);
+
+ if (CITime || log_is_enabled(Info, init)) {
+ CompilerStatistics* stats = nullptr;
+ if (task->is_scc()) {
+ int level = task->preload() ? CompLevel_full_optimization : (comp_level - 1);
+ stats = &_scc_stats_per_level[level];
+ } else {
+ stats = &_stats_per_level[comp_level-1];
+ }
+ stats->_invalidated.update(time, 0);
+ }
} else {
// Compilation succeeded
// update compilation ticks - used by the implementation of
// java.lang.management.CompilationMXBean
_perf_total_compilation->inc(time.ticks());
_peak_compilation_time = time.milliseconds() > _peak_compilation_time ? time.milliseconds() : _peak_compilation_time;
! if (CITime || log_is_enabled(Info, init)) {
int bytes_compiled = method->code_size() + task->num_inlined_bytecodes();
if (is_osr) {
! _t_osr_compilation.add(&time);
_sum_osr_bytes_compiled += bytes_compiled;
} else {
! _t_standard_compilation.add(&time);
_sum_standard_bytes_compiled += method->code_size() + task->num_inlined_bytecodes();
}
// Collect statistic per compilation level
! if (task->is_scc()) {
+ _scc_stats._standard.update(time, bytes_compiled);
+ _scc_stats._nmethods_size += task->nm_total_size();
+ _scc_stats._nmethods_code_size += task->nm_insts_size();
+ int level = task->preload() ? CompLevel_full_optimization : (comp_level - 1);
+ CompilerStatistics* stats = &_scc_stats_per_level[level];
+ stats->_standard.update(time, bytes_compiled);
+ stats->_nmethods_size += task->nm_total_size();
+ stats->_nmethods_code_size += task->nm_insts_size();
+ } else if (comp_level > CompLevel_none && comp_level <= CompLevel_full_optimization) {
CompilerStatistics* stats = &_stats_per_level[comp_level-1];
if (is_osr) {
stats->_osr.update(time, bytes_compiled);
} else {
stats->_standard.update(time, bytes_compiled);
} else {
assert(false, "CompilerStatistics object does not exist for compilation level %d", comp_level);
}
// Collect statistic per compiler
! AbstractCompiler* comp = compiler(comp_level);
! if (comp) {
CompilerStatistics* stats = comp->stats();
if (is_osr) {
stats->_osr.update(time, bytes_compiled);
} else {
stats->_standard.update(time, bytes_compiled);
}
stats->_nmethods_size += task->nm_total_size();
stats->_nmethods_code_size += task->nm_insts_size();
! } else { // if (!comp)
assert(false, "Compiler object must exist");
}
}
if (UsePerfData) {
} else {
assert(false, "CompilerStatistics object does not exist for compilation level %d", comp_level);
}
// Collect statistic per compiler
! AbstractCompiler* comp = task->compiler();
! if (comp && !task->is_scc()) {
CompilerStatistics* stats = comp->stats();
if (is_osr) {
stats->_osr.update(time, bytes_compiled);
} else {
stats->_standard.update(time, bytes_compiled);
}
stats->_nmethods_size += task->nm_total_size();
stats->_nmethods_code_size += task->nm_insts_size();
! } else if (!task->is_scc()) { // if (!comp)
assert(false, "Compiler object must exist");
}
}
if (UsePerfData) {
jlong CompileBroker::total_compilation_ticks() {
return _perf_total_compilation != nullptr ? _perf_total_compilation->get_value() : 0;
}
void CompileBroker::print_times(const char* name, CompilerStatistics* stats) {
tty->print_cr(" %s {speed: %6.3f bytes/s; standard: %6.3f s, %u bytes, %u methods; osr: %6.3f s, %u bytes, %u methods; nmethods_size: %u bytes; nmethods_code_size: %u bytes}",
name, stats->bytes_per_second(),
stats->_standard._time.seconds(), stats->_standard._bytes, stats->_standard._count,
stats->_osr._time.seconds(), stats->_osr._bytes, stats->_osr._count,
stats->_nmethods_size, stats->_nmethods_code_size);
}
void CompileBroker::print_times(bool per_compiler, bool aggregate) {
if (per_compiler) {
if (aggregate) {
tty->cr();
! tty->print_cr("Individual compiler times (for compiled methods only)");
tty->print_cr("------------------------------------------------");
tty->cr();
}
for (unsigned int i = 0; i < sizeof(_compilers) / sizeof(AbstractCompiler*); i++) {
AbstractCompiler* comp = _compilers[i];
if (comp != nullptr) {
print_times(comp->name(), comp->stats());
}
}
if (aggregate) {
tty->cr();
tty->print_cr("Individual compilation Tier times (for compiled methods only)");
tty->print_cr("------------------------------------------------");
tty->cr();
jlong CompileBroker::total_compilation_ticks() {
return _perf_total_compilation != nullptr ? _perf_total_compilation->get_value() : 0;
}
+ void CompileBroker::log_not_entrant(nmethod* nm) {
+ _total_not_entrant_count++;
+ if (CITime || log_is_enabled(Info, init)) {
+ CompilerStatistics* stats = nullptr;
+ int level = nm->comp_level();
+ if (nm->is_scc()) {
+ if (nm->preloaded()) {
+ assert(level == CompLevel_full_optimization, "%d", level);
+ level = CompLevel_full_optimization + 1;
+ }
+ stats = &_scc_stats_per_level[level - 1];
+ } else {
+ stats = &_stats_per_level[level - 1];
+ }
+ stats->_made_not_entrant._count++;
+ }
+ }
+
void CompileBroker::print_times(const char* name, CompilerStatistics* stats) {
tty->print_cr(" %s {speed: %6.3f bytes/s; standard: %6.3f s, %u bytes, %u methods; osr: %6.3f s, %u bytes, %u methods; nmethods_size: %u bytes; nmethods_code_size: %u bytes}",
name, stats->bytes_per_second(),
stats->_standard._time.seconds(), stats->_standard._bytes, stats->_standard._count,
stats->_osr._time.seconds(), stats->_osr._bytes, stats->_osr._count,
stats->_nmethods_size, stats->_nmethods_code_size);
}
+ static void print_helper(outputStream* st, const char* name, CompilerStatistics::Data data, bool print_time = true) {
+ if (data._count > 0) {
+ st->print("; %s: %4u methods", name, data._count);
+ if (print_time) {
+ st->print(" (in %.3fs)", data._time.seconds());
+ }
+ }
+ }
+
+ static void print_tier_helper(outputStream* st, const char* prefix, int tier, CompilerStatistics* stats) {
+ st->print(" %s%d: %5u methods", prefix, tier, stats->_standard._count);
+ if (stats->_standard._count > 0) {
+ st->print(" (in %.3fs)", stats->_standard._time.seconds());
+ }
+ print_helper(st, "osr", stats->_osr);
+ print_helper(st, "bailout", stats->_bailout);
+ print_helper(st, "invalid", stats->_invalidated);
+ print_helper(st, "not_entrant", stats->_made_not_entrant, false);
+ st->cr();
+ }
+
+ static void print_queue_info(outputStream* st, CompileQueue* queue) {
+ if (queue != nullptr) {
+ MutexLocker ml(queue->lock());
+
+ uint total_cnt = 0;
+ uint active_cnt = 0;
+ for (JavaThread* jt : *ThreadsSMRSupport::get_java_thread_list()) {
+ guarantee(jt != nullptr, "");
+ if (jt->is_Compiler_thread()) {
+ CompilerThread* ct = (CompilerThread*)jt;
+
+ guarantee(ct != nullptr, "");
+ if (ct->queue() == queue) {
+ ++total_cnt;
+ CompileTask* task = ct->task();
+ if (task != nullptr) {
+ ++active_cnt;
+ }
+ }
+ }
+ }
+
+ st->print(" %s (%d active / %d total threads): %u tasks",
+ queue->name(), active_cnt, total_cnt, queue->size());
+ if (queue->size() > 0) {
+ uint counts[] = {0, 0, 0, 0, 0}; // T1 ... T5
+ for (CompileTask* task = queue->first(); task != nullptr; task = task->next()) {
+ int tier = task->comp_level();
+ if (task->is_scc() && task->preload()) {
+ assert(tier == CompLevel_full_optimization, "%d", tier);
+ tier = CompLevel_full_optimization + 1;
+ }
+ counts[tier-1]++;
+ }
+ st->print(":");
+ for (int tier = CompLevel_simple; tier <= CompilationPolicy::highest_compile_level() + 1; tier++) {
+ uint cnt = counts[tier-1];
+ if (cnt > 0) {
+ st->print(" T%d: %u tasks;", tier, cnt);
+ }
+ }
+ }
+ st->cr();
+
+ // for (JavaThread* jt : *ThreadsSMRSupport::get_java_thread_list()) {
+ // guarantee(jt != nullptr, "");
+ // if (jt->is_Compiler_thread()) {
+ // CompilerThread* ct = (CompilerThread*)jt;
+ //
+ // guarantee(ct != nullptr, "");
+ // if (ct->queue() == queue) {
+ // ResourceMark rm;
+ // CompileTask* task = ct->task();
+ // st->print(" %s: ", ct->name_raw());
+ // if (task != nullptr) {
+ // task->print(st, nullptr, true /*short_form*/, false /*cr*/);
+ // }
+ // st->cr();
+ // }
+ // }
+ // }
+ }
+ }
+ void CompileBroker::print_statistics_on(outputStream* st) {
+ st->print_cr(" Total: %u methods; %u bailouts, %u invalidated, %u non_entrant",
+ _total_compile_count, _total_bailout_count, _total_invalidated_count, _total_not_entrant_count);
+ for (int tier = CompLevel_simple; tier <= CompilationPolicy::highest_compile_level(); tier++) {
+ print_tier_helper(st, "Tier", tier, &_stats_per_level[tier-1]);
+ }
+ st->cr();
+
+ if (LoadCachedCode || StoreCachedCode) {
+ for (int tier = CompLevel_simple; tier <= CompilationPolicy::highest_compile_level() + 1; tier++) {
+ if (tier != CompLevel_full_profile) {
+ print_tier_helper(st, "SC T", tier, &_scc_stats_per_level[tier - 1]);
+ }
+ }
+ st->cr();
+ }
+
+ print_queue_info(st, _c1_compile_queue);
+ print_queue_info(st, _c2_compile_queue);
+ print_queue_info(st, _c3_compile_queue);
+ print_queue_info(st, _sc1_compile_queue);
+ print_queue_info(st, _sc2_compile_queue);
+ }
+
void CompileBroker::print_times(bool per_compiler, bool aggregate) {
if (per_compiler) {
if (aggregate) {
tty->cr();
! tty->print_cr("[%dms] Individual compiler times (for compiled methods only)", (int)tty->time_stamp().milliseconds());
tty->print_cr("------------------------------------------------");
tty->cr();
}
for (unsigned int i = 0; i < sizeof(_compilers) / sizeof(AbstractCompiler*); i++) {
AbstractCompiler* comp = _compilers[i];
if (comp != nullptr) {
print_times(comp->name(), comp->stats());
}
}
+ if (_scc_stats._standard._count > 0) {
+ print_times("SC", &_scc_stats);
+ }
if (aggregate) {
tty->cr();
tty->print_cr("Individual compilation Tier times (for compiled methods only)");
tty->print_cr("------------------------------------------------");
tty->cr();
for (int tier = CompLevel_simple; tier <= CompilationPolicy::highest_compile_level(); tier++) {
CompilerStatistics* stats = &_stats_per_level[tier-1];
os::snprintf_checked(tier_name, sizeof(tier_name), "Tier%d", tier);
print_times(tier_name, stats);
}
+ for (int tier = CompLevel_simple; tier <= CompilationPolicy::highest_compile_level() + 1; tier++) {
+ CompilerStatistics* stats = &_scc_stats_per_level[tier-1];
+ if (stats->_standard._bytes > 0) {
+ os::snprintf_checked(tier_name, sizeof(tier_name), "SC T%d", tier);
+ print_times(tier_name, stats);
+ }
+ }
}
if (!aggregate) {
return;
}
osr_compile_count == 0 ? 0.0 : osr_compilation.seconds() / osr_compile_count);
tty->print_cr(" Invalidated : %7.3f s, Average : %2.3f s",
CompileBroker::_t_invalidated_compilation.seconds(),
total_invalidated_count == 0 ? 0.0 : CompileBroker::_t_invalidated_compilation.seconds() / total_invalidated_count);
+ if (StoreCachedCode || LoadCachedCode) { // Check flags because SC cache could be closed already
+ tty->cr();
+ SCCache::print_timers_on(tty);
+ }
AbstractCompiler *comp = compiler(CompLevel_simple);
if (comp != nullptr) {
tty->cr();
comp->print_timers();
}
comp = compiler(CompLevel_full_optimization);
if (comp != nullptr) {
tty->cr();
comp->print_timers();
}
+ comp = _compilers[2];
+ if (comp != nullptr) {
+ tty->cr();
+ comp->print_timers();
+ }
#if INCLUDE_JVMCI
if (EnableJVMCI) {
JVMCICompiler *jvmci_comp = JVMCICompiler::instance(false, JavaThread::current_or_null());
if (jvmci_comp != nullptr && jvmci_comp != comp) {
tty->cr();
< prev index next >