提交 2a9a8653 编写于 作者: A anoll

8040798: compiler/startup/SmallCodeCacheStartup.java timed out in RT_Baseline

Summary: Fixes broken memory freeing of compile queue tasks and makes sure that blocking compiles do not hang the VM if compilation gets disabled due to a full code cache.
Reviewed-by: kvn, iveresov
上级 f641d0b3
...@@ -183,9 +183,8 @@ int CompileBroker::_sum_nmethod_code_size = 0; ...@@ -183,9 +183,8 @@ int CompileBroker::_sum_nmethod_code_size = 0;
long CompileBroker::_peak_compilation_time = 0; long CompileBroker::_peak_compilation_time = 0;
CompileQueue* CompileBroker::_c2_method_queue = NULL; CompileQueue* CompileBroker::_c2_compile_queue = NULL;
CompileQueue* CompileBroker::_c1_method_queue = NULL; CompileQueue* CompileBroker::_c1_compile_queue = NULL;
CompileTask* CompileBroker::_task_free_list = NULL;
GrowableArray<CompilerThread*>* CompileBroker::_compiler_threads = NULL; GrowableArray<CompilerThread*>* CompileBroker::_compiler_threads = NULL;
...@@ -253,13 +252,56 @@ CompileTaskWrapper::~CompileTaskWrapper() { ...@@ -253,13 +252,56 @@ CompileTaskWrapper::~CompileTaskWrapper() {
// By convention, the compiling thread is responsible for // By convention, the compiling thread is responsible for
// recycling a non-blocking CompileTask. // recycling a non-blocking CompileTask.
CompileBroker::free_task(task); CompileTask::free(task);
} }
} }
// ------------------------------------------------------------------ CompileTask* CompileTask::_task_free_list = NULL;
// CompileTask::initialize #ifdef ASSERT
int CompileTask::_num_allocated_tasks = 0;
#endif
/**
* Allocate a CompileTask, from the free list if possible.
*/
CompileTask* CompileTask::allocate() {
MutexLocker locker(CompileTaskAlloc_lock);
CompileTask* task = NULL;
if (_task_free_list != NULL) {
task = _task_free_list;
_task_free_list = task->next();
task->set_next(NULL);
} else {
task = new CompileTask();
DEBUG_ONLY(_num_allocated_tasks++;)
assert (_num_allocated_tasks < 10000, "Leaking compilation tasks?");
task->set_next(NULL);
task->set_is_free(true);
}
assert(task->is_free(), "Task must be free.");
task->set_is_free(false);
return task;
}
/**
* Add a task to the free list.
*/
void CompileTask::free(CompileTask* task) {
MutexLocker locker(CompileTaskAlloc_lock);
if (!task->is_free()) {
task->set_code(NULL);
assert(!task->lock()->is_locked(), "Should not be locked when freed");
JNIHandles::destroy_global(task->_method_holder);
JNIHandles::destroy_global(task->_hot_method_holder);
task->set_is_free(true);
task->set_next(_task_free_list);
_task_free_list = task;
}
}
void CompileTask::initialize(int compile_id, void CompileTask::initialize(int compile_id,
methodHandle method, methodHandle method,
int osr_bci, int osr_bci,
...@@ -318,15 +360,6 @@ void CompileTask::set_code(nmethod* nm) { ...@@ -318,15 +360,6 @@ void CompileTask::set_code(nmethod* nm) {
if (nm == NULL) _code_handle = NULL; // drop the handle also if (nm == NULL) _code_handle = NULL; // drop the handle also
} }
// ------------------------------------------------------------------
// CompileTask::free
void CompileTask::free() {
set_code(NULL);
assert(!_lock->is_locked(), "Should not be locked when freed");
JNIHandles::destroy_global(_method_holder);
JNIHandles::destroy_global(_hot_method_holder);
}
void CompileTask::mark_on_stack() { void CompileTask::mark_on_stack() {
// Mark these methods as something redefine classes cannot remove. // Mark these methods as something redefine classes cannot remove.
...@@ -594,9 +627,12 @@ void CompileTask::log_task_done(CompileLog* log) { ...@@ -594,9 +627,12 @@ void CompileTask::log_task_done(CompileLog* log) {
// Add a CompileTask to a CompileQueue /**
* Add a CompileTask to a CompileQueue
*/
void CompileQueue::add(CompileTask* task) { void CompileQueue::add(CompileTask* task) {
assert(lock()->owned_by_self(), "must own lock"); assert(lock()->owned_by_self(), "must own lock");
assert(!CompileBroker::is_compilation_disabled_forever(), "Do not add task if compilation is turned off forever");
task->set_next(NULL); task->set_next(NULL);
task->set_prev(NULL); task->set_prev(NULL);
...@@ -618,9 +654,7 @@ void CompileQueue::add(CompileTask* task) { ...@@ -618,9 +654,7 @@ void CompileQueue::add(CompileTask* task) {
// Mark the method as being in the compile queue. // Mark the method as being in the compile queue.
task->method()->set_queued_for_compilation(); task->method()->set_queued_for_compilation();
if (CIPrintCompileQueue) { NOT_PRODUCT(print();)
print();
}
if (LogCompilation && xtty != NULL) { if (LogCompilation && xtty != NULL) {
task->log_task_queued(); task->log_task_queued();
...@@ -630,14 +664,19 @@ void CompileQueue::add(CompileTask* task) { ...@@ -630,14 +664,19 @@ void CompileQueue::add(CompileTask* task) {
lock()->notify_all(); lock()->notify_all();
} }
void CompileQueue::delete_all() { void CompileQueue::free_all() {
assert(lock()->owned_by_self(), "must own lock"); MutexLocker mu(lock());
if (_first != NULL) { if (_first != NULL) {
for (CompileTask* task = _first; task != NULL; task = task->next()) { for (CompileTask* task = _first; task != NULL; task = task->next()) {
delete task; // Wake up thread that blocks on the compile task.
task->lock()->notify();
// Puts task back on the freelist.
CompileTask::free(task);
} }
_first = NULL; _first = NULL;
} }
// Wake up all threads that block on the queue.
lock()->notify_all();
} }
// ------------------------------------------------------------------ // ------------------------------------------------------------------
...@@ -767,18 +806,24 @@ void CompileQueue::mark_on_stack() { ...@@ -767,18 +806,24 @@ void CompileQueue::mark_on_stack() {
} }
} }
// ------------------------------------------------------------------ #ifndef PRODUCT
// CompileQueue::print /**
* Print entire compilation queue.
*/
void CompileQueue::print() { void CompileQueue::print() {
tty->print_cr("Contents of %s", name()); if (CIPrintCompileQueue) {
tty->print_cr("----------------------"); ttyLocker ttyl;
CompileTask* task = _first; tty->print_cr("Contents of %s", name());
while (task != NULL) { tty->print_cr("----------------------");
task->print_line(); CompileTask* task = _first;
task = task->next(); while (task != NULL) {
task->print_line();
task = task->next();
}
tty->print_cr("----------------------");
} }
tty->print_cr("----------------------");
} }
#endif // PRODUCT
CompilerCounters::CompilerCounters(const char* thread_name, int instance, TRAPS) { CompilerCounters::CompilerCounters(const char* thread_name, int instance, TRAPS) {
...@@ -851,9 +896,6 @@ void CompileBroker::compilation_init() { ...@@ -851,9 +896,6 @@ void CompileBroker::compilation_init() {
_compilers[1] = new SharkCompiler(); _compilers[1] = new SharkCompiler();
#endif // SHARK #endif // SHARK
// Initialize the CompileTask free list
_task_free_list = NULL;
// Start the CompilerThreads // Start the CompilerThreads
init_compiler_threads(c1_count, c2_count); init_compiler_threads(c1_count, c2_count);
// totalTime performance counter is always created as it is required // totalTime performance counter is always created as it is required
...@@ -1046,11 +1088,11 @@ void CompileBroker::init_compiler_threads(int c1_compiler_count, int c2_compiler ...@@ -1046,11 +1088,11 @@ void CompileBroker::init_compiler_threads(int c1_compiler_count, int c2_compiler
#endif // !ZERO && !SHARK #endif // !ZERO && !SHARK
// Initialize the compilation queue // Initialize the compilation queue
if (c2_compiler_count > 0) { if (c2_compiler_count > 0) {
_c2_method_queue = new CompileQueue("C2MethodQueue", MethodCompileQueue_lock); _c2_compile_queue = new CompileQueue("C2 CompileQueue", MethodCompileQueue_lock);
_compilers[1]->set_num_compiler_threads(c2_compiler_count); _compilers[1]->set_num_compiler_threads(c2_compiler_count);
} }
if (c1_compiler_count > 0) { if (c1_compiler_count > 0) {
_c1_method_queue = new CompileQueue("C1MethodQueue", MethodCompileQueue_lock); _c1_compile_queue = new CompileQueue("C1 CompileQueue", MethodCompileQueue_lock);
_compilers[0]->set_num_compiler_threads(c1_compiler_count); _compilers[0]->set_num_compiler_threads(c1_compiler_count);
} }
...@@ -1065,7 +1107,7 @@ void CompileBroker::init_compiler_threads(int c1_compiler_count, int c2_compiler ...@@ -1065,7 +1107,7 @@ void CompileBroker::init_compiler_threads(int c1_compiler_count, int c2_compiler
sprintf(name_buffer, "C2 CompilerThread%d", i); sprintf(name_buffer, "C2 CompilerThread%d", i);
CompilerCounters* counters = new CompilerCounters("compilerThread", i, CHECK); CompilerCounters* counters = new CompilerCounters("compilerThread", i, CHECK);
// Shark and C2 // Shark and C2
CompilerThread* new_thread = make_compiler_thread(name_buffer, _c2_method_queue, counters, _compilers[1], CHECK); CompilerThread* new_thread = make_compiler_thread(name_buffer, _c2_compile_queue, counters, _compilers[1], CHECK);
_compiler_threads->append(new_thread); _compiler_threads->append(new_thread);
} }
...@@ -1074,7 +1116,7 @@ void CompileBroker::init_compiler_threads(int c1_compiler_count, int c2_compiler ...@@ -1074,7 +1116,7 @@ void CompileBroker::init_compiler_threads(int c1_compiler_count, int c2_compiler
sprintf(name_buffer, "C1 CompilerThread%d", i); sprintf(name_buffer, "C1 CompilerThread%d", i);
CompilerCounters* counters = new CompilerCounters("compilerThread", i, CHECK); CompilerCounters* counters = new CompilerCounters("compilerThread", i, CHECK);
// C1 // C1
CompilerThread* new_thread = make_compiler_thread(name_buffer, _c1_method_queue, counters, _compilers[0], CHECK); CompilerThread* new_thread = make_compiler_thread(name_buffer, _c1_compile_queue, counters, _compilers[0], CHECK);
_compiler_threads->append(new_thread); _compiler_threads->append(new_thread);
} }
...@@ -1084,14 +1126,18 @@ void CompileBroker::init_compiler_threads(int c1_compiler_count, int c2_compiler ...@@ -1084,14 +1126,18 @@ void CompileBroker::init_compiler_threads(int c1_compiler_count, int c2_compiler
} }
// Set the methods on the stack as on_stack so that redefine classes doesn't /**
// reclaim them * Set the methods on the stack as on_stack so that redefine classes doesn't
* reclaim them
*/
void CompileBroker::mark_on_stack() { void CompileBroker::mark_on_stack() {
if (_c2_method_queue != NULL) { if (_c2_compile_queue != NULL) {
_c2_method_queue->mark_on_stack(); MutexLocker locker(_c2_compile_queue->lock());
_c2_compile_queue->mark_on_stack();
} }
if (_c1_method_queue != NULL) { if (_c1_compile_queue != NULL) {
_c1_method_queue->mark_on_stack(); MutexLocker locker(_c1_compile_queue->lock());
_c1_compile_queue->mark_on_stack();
} }
} }
...@@ -1107,7 +1153,7 @@ void CompileBroker::compile_method_base(methodHandle method, ...@@ -1107,7 +1153,7 @@ void CompileBroker::compile_method_base(methodHandle method,
const char* comment, const char* comment,
Thread* thread) { Thread* thread) {
// do nothing if compiler thread(s) is not available // do nothing if compiler thread(s) is not available
if (!_initialized ) { if (!_initialized) {
return; return;
} }
...@@ -1154,7 +1200,7 @@ void CompileBroker::compile_method_base(methodHandle method, ...@@ -1154,7 +1200,7 @@ void CompileBroker::compile_method_base(methodHandle method,
// If this method is already in the compile queue, then // If this method is already in the compile queue, then
// we do not block the current thread. // we do not block the current thread.
if (compilation_is_in_queue(method, osr_bci)) { if (compilation_is_in_queue(method)) {
// We may want to decay our counter a bit here to prevent // We may want to decay our counter a bit here to prevent
// multiple denied requests for compilation. This is an // multiple denied requests for compilation. This is an
// open compilation policy issue. Note: The other possibility, // open compilation policy issue. Note: The other possibility,
...@@ -1193,7 +1239,7 @@ void CompileBroker::compile_method_base(methodHandle method, ...@@ -1193,7 +1239,7 @@ void CompileBroker::compile_method_base(methodHandle method,
// Make sure the method has not slipped into the queues since // Make sure the method has not slipped into the queues since
// last we checked; note that those checks were "fast bail-outs". // last we checked; note that those checks were "fast bail-outs".
// Here we need to be more careful, see 14012000 below. // Here we need to be more careful, see 14012000 below.
if (compilation_is_in_queue(method, osr_bci)) { if (compilation_is_in_queue(method)) {
return; return;
} }
...@@ -1214,7 +1260,7 @@ void CompileBroker::compile_method_base(methodHandle method, ...@@ -1214,7 +1260,7 @@ void CompileBroker::compile_method_base(methodHandle method,
} }
// Should this thread wait for completion of the compile? // Should this thread wait for completion of the compile?
blocking = is_compile_blocking(method, osr_bci); blocking = is_compile_blocking();
// We will enter the compilation in the queue. // We will enter the compilation in the queue.
// 14012000: Note that this sets the queued_for_compile bits in // 14012000: Note that this sets the queued_for_compile bits in
...@@ -1406,19 +1452,17 @@ bool CompileBroker::compilation_is_complete(methodHandle method, ...@@ -1406,19 +1452,17 @@ bool CompileBroker::compilation_is_complete(methodHandle method,
} }
// ------------------------------------------------------------------ /**
// CompileBroker::compilation_is_in_queue * See if this compilation is already requested.
// *
// See if this compilation is already requested. * Implementation note: there is only a single "is in queue" bit
// * for each method. This means that the check below is overly
// Implementation note: there is only a single "is in queue" bit * conservative in the sense that an osr compilation in the queue
// for each method. This means that the check below is overly * will block a normal compilation from entering the queue (and vice
// conservative in the sense that an osr compilation in the queue * versa). This can be remedied by a full queue search to disambiguate
// will block a normal compilation from entering the queue (and vice * cases. If it is deemed profitable, this may be done.
// versa). This can be remedied by a full queue search to disambiguate */
// cases. If it is deemed profitible, this may be done. bool CompileBroker::compilation_is_in_queue(methodHandle method) {
bool CompileBroker::compilation_is_in_queue(methodHandle method,
int osr_bci) {
return method->queued_for_compilation(); return method->queued_for_compilation();
} }
...@@ -1498,13 +1542,11 @@ int CompileBroker::assign_compile_id(methodHandle method, int osr_bci) { ...@@ -1498,13 +1542,11 @@ int CompileBroker::assign_compile_id(methodHandle method, int osr_bci) {
#endif #endif
} }
/**
// ------------------------------------------------------------------ * Should the current thread block until this compilation request
// CompileBroker::is_compile_blocking * has been fulfilled?
// */
// Should the current thread be blocked until this compilation request bool CompileBroker::is_compile_blocking() {
// has been fulfilled?
bool CompileBroker::is_compile_blocking(methodHandle method, int osr_bci) {
assert(!InstanceRefKlass::owns_pending_list_lock(JavaThread::current()), "possible deadlock"); assert(!InstanceRefKlass::owns_pending_list_lock(JavaThread::current()), "possible deadlock");
return !BackgroundCompilation; return !BackgroundCompilation;
} }
...@@ -1532,7 +1574,7 @@ CompileTask* CompileBroker::create_compile_task(CompileQueue* queue, ...@@ -1532,7 +1574,7 @@ CompileTask* CompileBroker::create_compile_task(CompileQueue* queue,
int hot_count, int hot_count,
const char* comment, const char* comment,
bool blocking) { bool blocking) {
CompileTask* new_task = allocate_task(); CompileTask* new_task = CompileTask::allocate();
new_task->initialize(compile_id, method, osr_bci, comp_level, new_task->initialize(compile_id, method, osr_bci, comp_level,
hot_method, hot_count, comment, hot_method, hot_count, comment,
blocking); blocking);
...@@ -1541,75 +1583,52 @@ CompileTask* CompileBroker::create_compile_task(CompileQueue* queue, ...@@ -1541,75 +1583,52 @@ CompileTask* CompileBroker::create_compile_task(CompileQueue* queue,
} }
// ------------------------------------------------------------------ /**
// CompileBroker::allocate_task * Wait for the compilation task to complete.
// */
// Allocate a CompileTask, from the free list if possible.
CompileTask* CompileBroker::allocate_task() {
MutexLocker locker(CompileTaskAlloc_lock);
CompileTask* task = NULL;
if (_task_free_list != NULL) {
task = _task_free_list;
_task_free_list = task->next();
task->set_next(NULL);
} else {
task = new CompileTask();
task->set_next(NULL);
}
return task;
}
// ------------------------------------------------------------------
// CompileBroker::free_task
//
// Add a task to the free list.
void CompileBroker::free_task(CompileTask* task) {
MutexLocker locker(CompileTaskAlloc_lock);
task->free();
task->set_next(_task_free_list);
_task_free_list = task;
}
// ------------------------------------------------------------------
// CompileBroker::wait_for_completion
//
// Wait for the given method CompileTask to complete.
void CompileBroker::wait_for_completion(CompileTask* task) { void CompileBroker::wait_for_completion(CompileTask* task) {
if (CIPrintCompileQueue) { if (CIPrintCompileQueue) {
ttyLocker ttyl;
tty->print_cr("BLOCKING FOR COMPILE"); tty->print_cr("BLOCKING FOR COMPILE");
} }
assert(task->is_blocking(), "can only wait on blocking task"); assert(task->is_blocking(), "can only wait on blocking task");
JavaThread *thread = JavaThread::current(); JavaThread* thread = JavaThread::current();
thread->set_blocked_on_compilation(true); thread->set_blocked_on_compilation(true);
methodHandle method(thread, task->method()); methodHandle method(thread, task->method());
{ {
MutexLocker waiter(task->lock(), thread); MutexLocker waiter(task->lock(), thread);
while (!task->is_complete()) while (!task->is_complete() && !is_compilation_disabled_forever()) {
task->lock()->wait(); task->lock()->wait();
}
} }
thread->set_blocked_on_compilation(false);
if (is_compilation_disabled_forever()) {
CompileTask::free(task);
return;
}
// It is harmless to check this status without the lock, because // It is harmless to check this status without the lock, because
// completion is a stable property (until the task object is recycled). // completion is a stable property (until the task object is recycled).
assert(task->is_complete(), "Compilation should have completed"); assert(task->is_complete(), "Compilation should have completed");
assert(task->code_handle() == NULL, "must be reset"); assert(task->code_handle() == NULL, "must be reset");
thread->set_blocked_on_compilation(false);
// By convention, the waiter is responsible for recycling a // By convention, the waiter is responsible for recycling a
// blocking CompileTask. Since there is only one waiter ever // blocking CompileTask. Since there is only one waiter ever
// waiting on a CompileTask, we know that no one else will // waiting on a CompileTask, we know that no one else will
// be using this CompileTask; we can free it. // be using this CompileTask; we can free it.
free_task(task); CompileTask::free(task);
} }
// Initialize compiler thread(s) + compiler object(s). The postcondition /**
// of this function is that the compiler runtimes are initialized and that * Initialize compiler thread(s) + compiler object(s). The postcondition
//compiler threads can start compiling. * of this function is that the compiler runtimes are initialized and that
* compiler threads can start compiling.
*/
bool CompileBroker::init_compiler_runtime() { bool CompileBroker::init_compiler_runtime() {
CompilerThread* thread = CompilerThread::current(); CompilerThread* thread = CompilerThread::current();
AbstractCompiler* comp = thread->compiler(); AbstractCompiler* comp = thread->compiler();
...@@ -1646,7 +1665,6 @@ bool CompileBroker::init_compiler_runtime() { ...@@ -1646,7 +1665,6 @@ bool CompileBroker::init_compiler_runtime() {
disable_compilation_forever(); disable_compilation_forever();
// If compiler initialization failed, no compiler thread that is specific to a // If compiler initialization failed, no compiler thread that is specific to a
// particular compiler runtime will ever start to compile methods. // particular compiler runtime will ever start to compile methods.
shutdown_compiler_runtime(comp, thread); shutdown_compiler_runtime(comp, thread);
return false; return false;
} }
...@@ -1660,9 +1678,11 @@ bool CompileBroker::init_compiler_runtime() { ...@@ -1660,9 +1678,11 @@ bool CompileBroker::init_compiler_runtime() {
return true; return true;
} }
// If C1 and/or C2 initialization failed, we shut down all compilation. /**
// We do this to keep things simple. This can be changed if it ever turns out to be * If C1 and/or C2 initialization failed, we shut down all compilation.
// a problem. * We do this to keep things simple. This can be changed if it ever turns
* out to be a problem.
*/
void CompileBroker::shutdown_compiler_runtime(AbstractCompiler* comp, CompilerThread* thread) { void CompileBroker::shutdown_compiler_runtime(AbstractCompiler* comp, CompilerThread* thread) {
// Free buffer blob, if allocated // Free buffer blob, if allocated
if (thread->get_buffer_blob() != NULL) { if (thread->get_buffer_blob() != NULL) {
...@@ -1674,28 +1694,25 @@ void CompileBroker::shutdown_compiler_runtime(AbstractCompiler* comp, CompilerTh ...@@ -1674,28 +1694,25 @@ void CompileBroker::shutdown_compiler_runtime(AbstractCompiler* comp, CompilerTh
// There are two reasons for shutting down the compiler // There are two reasons for shutting down the compiler
// 1) compiler runtime initialization failed // 1) compiler runtime initialization failed
// 2) The code cache is full and the following flag is set: -XX:-UseCodeCacheFlushing // 2) The code cache is full and the following flag is set: -XX:-UseCodeCacheFlushing
warning("Shutting down compiler %s (no space to run compilers)", comp->name()); warning("%s initialization failed. Shutting down all compilers", comp->name());
// Only one thread per compiler runtime object enters here // Only one thread per compiler runtime object enters here
// Set state to shut down // Set state to shut down
comp->set_shut_down(); comp->set_shut_down();
MutexLocker mu(MethodCompileQueue_lock, thread); // Delete all queued compilation tasks to make compiler threads exit faster.
CompileQueue* queue; if (_c1_compile_queue != NULL) {
if (_c1_method_queue != NULL) { _c1_compile_queue->free_all();
_c1_method_queue->delete_all();
queue = _c1_method_queue;
_c1_method_queue = NULL;
delete _c1_method_queue;
} }
if (_c2_method_queue != NULL) { if (_c2_compile_queue != NULL) {
_c2_method_queue->delete_all(); _c2_compile_queue->free_all();
queue = _c2_method_queue;
_c2_method_queue = NULL;
delete _c2_method_queue;
} }
// 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 // We could delete compiler runtimes also. However, there are references to
// the compiler runtime(s) (e.g., nmethod::is_compiled_by_c1()) which then // the compiler runtime(s) (e.g., nmethod::is_compiled_by_c1()) which then
// fail. This can be done later if necessary. // fail. This can be done later if necessary.
......
...@@ -40,6 +40,11 @@ class CompileTask : public CHeapObj<mtCompiler> { ...@@ -40,6 +40,11 @@ class CompileTask : public CHeapObj<mtCompiler> {
friend class VMStructs; friend class VMStructs;
private: private:
static CompileTask* _task_free_list;
#ifdef ASSERT
static int _num_allocated_tasks;
#endif
Monitor* _lock; Monitor* _lock;
uint _compile_id; uint _compile_id;
Method* _method; Method* _method;
...@@ -52,7 +57,7 @@ class CompileTask : public CHeapObj<mtCompiler> { ...@@ -52,7 +57,7 @@ class CompileTask : public CHeapObj<mtCompiler> {
int _num_inlined_bytecodes; int _num_inlined_bytecodes;
nmethodLocker* _code_handle; // holder of eventual result nmethodLocker* _code_handle; // holder of eventual result
CompileTask* _next, *_prev; CompileTask* _next, *_prev;
bool _is_free;
// Fields used for logging why the compilation was initiated: // Fields used for logging why the compilation was initiated:
jlong _time_queued; // in units of os::elapsed_counter() jlong _time_queued; // in units of os::elapsed_counter()
Method* _hot_method; // which method actually triggered this task Method* _hot_method; // which method actually triggered this task
...@@ -70,7 +75,8 @@ class CompileTask : public CHeapObj<mtCompiler> { ...@@ -70,7 +75,8 @@ class CompileTask : public CHeapObj<mtCompiler> {
methodHandle hot_method, int hot_count, const char* comment, methodHandle hot_method, int hot_count, const char* comment,
bool is_blocking); bool is_blocking);
void free(); static CompileTask* allocate();
static void free(CompileTask* task);
int compile_id() const { return _compile_id; } int compile_id() const { return _compile_id; }
Method* method() const { return _method; } Method* method() const { return _method; }
...@@ -99,6 +105,8 @@ class CompileTask : public CHeapObj<mtCompiler> { ...@@ -99,6 +105,8 @@ class CompileTask : public CHeapObj<mtCompiler> {
void set_next(CompileTask* next) { _next = next; } void set_next(CompileTask* next) { _next = next; }
CompileTask* prev() const { return _prev; } CompileTask* prev() const { return _prev; }
void set_prev(CompileTask* prev) { _prev = prev; } void set_prev(CompileTask* prev) { _prev = prev; }
bool is_free() const { return _is_free; }
void set_is_free(bool val) { _is_free = val; }
private: private:
static void print_compilation_impl(outputStream* st, Method* method, int compile_id, int comp_level, static void print_compilation_impl(outputStream* st, Method* method, int compile_id, int comp_level,
...@@ -225,8 +233,8 @@ class CompileQueue : public CHeapObj<mtCompiler> { ...@@ -225,8 +233,8 @@ class CompileQueue : public CHeapObj<mtCompiler> {
// Redefine Classes support // Redefine Classes support
void mark_on_stack(); void mark_on_stack();
void delete_all(); void free_all();
void print(); NOT_PRODUCT (void print();)
~CompileQueue() { ~CompileQueue() {
assert (is_empty(), " Compile Queue must be empty"); assert (is_empty(), " Compile Queue must be empty");
...@@ -279,9 +287,8 @@ class CompileBroker: AllStatic { ...@@ -279,9 +287,8 @@ class CompileBroker: AllStatic {
static int _last_compile_level; static int _last_compile_level;
static char _last_method_compiled[name_buffer_length]; static char _last_method_compiled[name_buffer_length];
static CompileQueue* _c2_method_queue; static CompileQueue* _c2_compile_queue;
static CompileQueue* _c1_method_queue; static CompileQueue* _c1_compile_queue;
static CompileTask* _task_free_list;
static GrowableArray<CompilerThread*>* _compiler_threads; static GrowableArray<CompilerThread*>* _compiler_threads;
...@@ -334,7 +341,7 @@ class CompileBroker: AllStatic { ...@@ -334,7 +341,7 @@ class CompileBroker: AllStatic {
static void init_compiler_threads(int c1_compiler_count, int c2_compiler_count); static void init_compiler_threads(int c1_compiler_count, int c2_compiler_count);
static bool compilation_is_complete (methodHandle method, int osr_bci, int comp_level); static bool compilation_is_complete (methodHandle method, int osr_bci, int comp_level);
static bool compilation_is_prohibited(methodHandle method, int osr_bci, int comp_level); static bool compilation_is_prohibited(methodHandle method, int osr_bci, int comp_level);
static bool is_compile_blocking (methodHandle method, int osr_bci); static bool is_compile_blocking ();
static void preload_classes (methodHandle method, TRAPS); static void preload_classes (methodHandle method, TRAPS);
static CompileTask* create_compile_task(CompileQueue* queue, static CompileTask* create_compile_task(CompileQueue* queue,
...@@ -346,8 +353,6 @@ class CompileBroker: AllStatic { ...@@ -346,8 +353,6 @@ class CompileBroker: AllStatic {
int hot_count, int hot_count,
const char* comment, const char* comment,
bool blocking); bool blocking);
static CompileTask* allocate_task();
static void free_task(CompileTask* task);
static void wait_for_completion(CompileTask* task); static void wait_for_completion(CompileTask* task);
static void invoke_compiler_on_method(CompileTask* task); static void invoke_compiler_on_method(CompileTask* task);
...@@ -365,8 +370,8 @@ class CompileBroker: AllStatic { ...@@ -365,8 +370,8 @@ class CompileBroker: AllStatic {
const char* comment, const char* comment,
Thread* thread); Thread* thread);
static CompileQueue* compile_queue(int comp_level) { static CompileQueue* compile_queue(int comp_level) {
if (is_c2_compile(comp_level)) return _c2_method_queue; if (is_c2_compile(comp_level)) return _c2_compile_queue;
if (is_c1_compile(comp_level)) return _c1_method_queue; if (is_c1_compile(comp_level)) return _c1_compile_queue;
return NULL; return NULL;
} }
static bool init_compiler_runtime(); static bool init_compiler_runtime();
...@@ -384,7 +389,7 @@ class CompileBroker: AllStatic { ...@@ -384,7 +389,7 @@ class CompileBroker: AllStatic {
return NULL; return NULL;
} }
static bool compilation_is_in_queue(methodHandle method, int osr_bci); static bool compilation_is_in_queue(methodHandle method);
static int queue_size(int comp_level) { static int queue_size(int comp_level) {
CompileQueue *q = compile_queue(comp_level); CompileQueue *q = compile_queue(comp_level);
return q != NULL ? q->size() : 0; return q != NULL ? q->size() : 0;
......
...@@ -451,7 +451,7 @@ void AdvancedThresholdPolicy::method_invocation_event(methodHandle mh, methodHan ...@@ -451,7 +451,7 @@ void AdvancedThresholdPolicy::method_invocation_event(methodHandle mh, methodHan
if (should_create_mdo(mh(), level)) { if (should_create_mdo(mh(), level)) {
create_mdo(mh, thread); create_mdo(mh, thread);
} }
if (is_compilation_enabled() && !CompileBroker::compilation_is_in_queue(mh, InvocationEntryBci)) { if (is_compilation_enabled() && !CompileBroker::compilation_is_in_queue(mh)) {
CompLevel next_level = call_event(mh(), level); CompLevel next_level = call_event(mh(), level);
if (next_level != level) { if (next_level != level) {
compile(mh, InvocationEntryBci, next_level, thread); compile(mh, InvocationEntryBci, next_level, thread);
...@@ -475,7 +475,7 @@ void AdvancedThresholdPolicy::method_back_branch_event(methodHandle mh, methodHa ...@@ -475,7 +475,7 @@ void AdvancedThresholdPolicy::method_back_branch_event(methodHandle mh, methodHa
CompLevel next_osr_level = loop_event(imh(), level); CompLevel next_osr_level = loop_event(imh(), level);
CompLevel max_osr_level = (CompLevel)imh->highest_osr_comp_level(); CompLevel max_osr_level = (CompLevel)imh->highest_osr_comp_level();
// At the very least compile the OSR version // At the very least compile the OSR version
if (!CompileBroker::compilation_is_in_queue(imh, bci) && next_osr_level != level) { if (!CompileBroker::compilation_is_in_queue(imh) && (next_osr_level != level)) {
compile(imh, bci, next_osr_level, thread); compile(imh, bci, next_osr_level, thread);
} }
...@@ -509,7 +509,7 @@ void AdvancedThresholdPolicy::method_back_branch_event(methodHandle mh, methodHa ...@@ -509,7 +509,7 @@ void AdvancedThresholdPolicy::method_back_branch_event(methodHandle mh, methodHa
nm->make_not_entrant(); nm->make_not_entrant();
} }
} }
if (!CompileBroker::compilation_is_in_queue(mh, InvocationEntryBci)) { if (!CompileBroker::compilation_is_in_queue(mh)) {
// Fix up next_level if necessary to avoid deopts // Fix up next_level if necessary to avoid deopts
if (next_level == CompLevel_limited_profile && max_osr_level == CompLevel_full_profile) { if (next_level == CompLevel_limited_profile && max_osr_level == CompLevel_full_profile) {
next_level = CompLevel_full_profile; next_level = CompLevel_full_profile;
...@@ -521,7 +521,7 @@ void AdvancedThresholdPolicy::method_back_branch_event(methodHandle mh, methodHa ...@@ -521,7 +521,7 @@ void AdvancedThresholdPolicy::method_back_branch_event(methodHandle mh, methodHa
} else { } else {
cur_level = comp_level(imh()); cur_level = comp_level(imh());
next_level = call_event(imh(), cur_level); next_level = call_event(imh(), cur_level);
if (!CompileBroker::compilation_is_in_queue(imh, bci) && next_level != cur_level) { if (!CompileBroker::compilation_is_in_queue(imh) && (next_level != cur_level)) {
compile(imh, InvocationEntryBci, next_level, thread); compile(imh, InvocationEntryBci, next_level, thread);
} }
} }
......
...@@ -239,7 +239,7 @@ void SimpleThresholdPolicy::compile(methodHandle mh, int bci, CompLevel level, J ...@@ -239,7 +239,7 @@ void SimpleThresholdPolicy::compile(methodHandle mh, int bci, CompLevel level, J
if (bci != InvocationEntryBci && mh->is_not_osr_compilable(level)) { if (bci != InvocationEntryBci && mh->is_not_osr_compilable(level)) {
return; return;
} }
if (!CompileBroker::compilation_is_in_queue(mh, bci)) { if (!CompileBroker::compilation_is_in_queue(mh)) {
if (PrintTieredEvents) { if (PrintTieredEvents) {
print_event(COMPILE, mh, mh, bci, level); print_event(COMPILE, mh, mh, bci, level);
} }
...@@ -378,7 +378,7 @@ CompLevel SimpleThresholdPolicy::loop_event(Method* method, CompLevel cur_level) ...@@ -378,7 +378,7 @@ CompLevel SimpleThresholdPolicy::loop_event(Method* method, CompLevel cur_level)
// Handle the invocation event. // Handle the invocation event.
void SimpleThresholdPolicy::method_invocation_event(methodHandle mh, methodHandle imh, void SimpleThresholdPolicy::method_invocation_event(methodHandle mh, methodHandle imh,
CompLevel level, nmethod* nm, JavaThread* thread) { CompLevel level, nmethod* nm, JavaThread* thread) {
if (is_compilation_enabled() && !CompileBroker::compilation_is_in_queue(mh, InvocationEntryBci)) { if (is_compilation_enabled() && !CompileBroker::compilation_is_in_queue(mh)) {
CompLevel next_level = call_event(mh(), level); CompLevel next_level = call_event(mh(), level);
if (next_level != level) { if (next_level != level) {
compile(mh, InvocationEntryBci, next_level, thread); compile(mh, InvocationEntryBci, next_level, thread);
...@@ -391,8 +391,8 @@ void SimpleThresholdPolicy::method_invocation_event(methodHandle mh, methodHandl ...@@ -391,8 +391,8 @@ void SimpleThresholdPolicy::method_invocation_event(methodHandle mh, methodHandl
void SimpleThresholdPolicy::method_back_branch_event(methodHandle mh, methodHandle imh, void SimpleThresholdPolicy::method_back_branch_event(methodHandle mh, methodHandle imh,
int bci, CompLevel level, nmethod* nm, JavaThread* thread) { int bci, CompLevel level, nmethod* nm, JavaThread* thread) {
// If the method is already compiling, quickly bail out. // If the method is already compiling, quickly bail out.
if (is_compilation_enabled() && !CompileBroker::compilation_is_in_queue(mh, bci)) { if (is_compilation_enabled() && !CompileBroker::compilation_is_in_queue(mh)) {
// Use loop event as an opportinity to also check there's been // Use loop event as an opportunity to also check there's been
// enough calls. // enough calls.
CompLevel cur_level = comp_level(mh()); CompLevel cur_level = comp_level(mh());
CompLevel next_level = call_event(mh(), cur_level); CompLevel next_level = call_event(mh(), cur_level);
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册