< prev index next > src/hotspot/share/runtime/synchronizer.cpp
Print this page
*/
#include "precompiled.hpp"
#include "classfile/vmSymbols.hpp"
#include "jfr/jfrEvents.hpp"
+ #include "gc/shared/suspendibleThreadSet.hpp"
#include "logging/log.hpp"
#include "logging/logStream.hpp"
#include "memory/allocation.inline.hpp"
#include "memory/padded.hpp"
#include "memory/resourceArea.hpp"
#include "runtime/atomic.hpp"
#include "runtime/biasedLocking.hpp"
#include "runtime/handles.inline.hpp"
#include "runtime/handshake.hpp"
#include "runtime/interfaceSupport.inline.hpp"
+ #include "runtime/lockStack.inline.hpp"
#include "runtime/mutexLocker.hpp"
#include "runtime/objectMonitor.hpp"
#include "runtime/objectMonitor.inline.hpp"
#include "runtime/os.inline.hpp"
#include "runtime/osThread.hpp"
assert(current->thread_state() == _thread_in_Java, "invariant");
NoSafepointVerifier nsv;
if (obj == NULL) return false; // slow-path for invalid obj
const markWord mark = obj->mark();
- if (mark.has_locker() && current->is_lock_owned((address)mark.locker())) {
- // Degenerate notify
- // stack-locked by caller so by definition the implied waitset is empty.
- return true;
+ if (LockingMode == LM_LIGHTWEIGHT) {
+ if (mark.is_fast_locked() && current->lock_stack().contains(cast_to_oop(obj))) {
+ // Degenerate notify
+ // fast-locked by caller so by definition the implied waitset is empty.
+ return true;
+ }
+ } else if (LockingMode == LM_LEGACY) {
+ if (mark.has_locker() && current->is_lock_owned((address)mark.locker())) {
+ // Degenerate notify
+ // stack-locked by caller so by definition the implied waitset is empty.
+ return true;
+ }
}
if (mark.has_monitor()) {
ObjectMonitor* const mon = mark.monitor();
assert(mon->object() == oop(obj), "invariant");
if (owner == current) {
m->_recursions++;
return true;
}
- // This Java Monitor is inflated so obj's header will never be
- // displaced to this thread's BasicLock. Make the displaced header
- // non-NULL so this BasicLock is not seen as recursive nor as
- // being locked. We do this unconditionally so that this thread's
- // BasicLock cannot be mis-interpreted by any stack walkers. For
- // performance reasons, stack walkers generally first check for
- // Biased Locking in the object's header, the second check is for
- // stack-locking in the object's header, the third check is for
- // recursive stack-locking in the displaced header in the BasicLock,
- // and last are the inflated Java Monitor (ObjectMonitor) checks.
- lock->set_displaced_header(markWord::unused_mark());
+ if (LockingMode != LM_LIGHTWEIGHT) {
+ // This Java Monitor is inflated so obj's header will never be
+ // displaced to this thread's BasicLock. Make the displaced header
+ // non-NULL so this BasicLock is not seen as recursive nor as
+ // being locked. We do this unconditionally so that this thread's
+ // BasicLock cannot be mis-interpreted by any stack walkers. For
+ // performance reasons, stack walkers generally first check for
+ // Biased Locking in the object's header, the second check is for
+ // stack-locking in the object's header, the third check is for
+ // recursive stack-locking in the displaced header in the BasicLock,
+ // and last are the inflated Java Monitor (ObjectMonitor) checks.
+ lock->set_displaced_header(markWord::unused_mark());
+ }
if (owner == NULL && m->try_set_owner_from(NULL, current) == NULL) {
assert(m->_recursions == 0, "invariant");
return true;
}
if (bcp_was_adjusted) {
last_frame.interpreter_frame_set_bcp(last_frame.interpreter_frame_bcp() + 1);
}
}
+ static bool useHeavyMonitors() {
+ #if defined(X86) || defined(AARCH64) || defined(PPC64) || defined(RISCV64)
+ return LockingMode == LM_MONITOR;
+ #else
+ return false;
+ #endif
+ }
+
// -----------------------------------------------------------------------------
// Monitor Enter/Exit
// The interpreter and compiler assembly code tries to lock using the fast path
// of this algorithm. Make sure to update that code if the following function is
// changed. The implementation is extremely sensitive to race condition. Be careful.
void ObjectSynchronizer::enter(Handle obj, BasicLock* lock, JavaThread* current) {
if (obj->klass()->is_value_based()) {
handle_sync_on_value_based_class(obj, current);
}
- if (UseBiasedLocking) {
- BiasedLocking::revoke(current, obj);
- }
+ if (!useHeavyMonitors()) {
+ if (LockingMode == LM_LIGHTWEIGHT) {
+ // Fast-locking does not use the 'lock' argument.
+ LockStack& lock_stack = current->lock_stack();
+ if (lock_stack.can_push()) {
+ markWord mark = obj()->mark_acquire();
+ if (mark.is_neutral()) {
+ assert(!lock_stack.contains(obj()), "thread must not already hold the lock");
+ // Try to swing into 'fast-locked' state.
+ markWord locked_mark = mark.set_fast_locked();
+ markWord old_mark = obj()->cas_set_mark(locked_mark, mark);
+ if (old_mark == mark) {
+ // Successfully fast-locked, push object to lock-stack and return.
+ lock_stack.push(obj());
+ return;
+ }
+ }
+ }
+ // All other paths fall-through to inflate-enter.
+ } else if (LockingMode == LM_LEGACY) {
+ if (UseBiasedLocking) {
+ BiasedLocking::revoke(current, obj);
+ }
- markWord mark = obj->mark();
- assert(!mark.has_bias_pattern(), "should not see bias pattern here");
+ markWord mark = obj->mark();
+ if (mark.is_neutral()) {
+ // Anticipate successful CAS -- the ST of the displaced mark must
+ // be visible <= the ST performed by the CAS.
+ lock->set_displaced_header(mark);
+ if (mark == obj()->cas_set_mark(markWord::from_pointer(lock), mark)) {
+ return;
+ }
+ // Fall through to inflate() ...
+ } else if (mark.has_locker() &&
+ current->is_lock_owned((address) mark.locker())) {
+ assert(lock != mark.locker(), "must not re-lock the same lock");
+ assert(lock != (BasicLock*) obj->mark().value(), "don't relock with same BasicLock");
+ lock->set_displaced_header(markWord::from_pointer(NULL));
+ return;
+ }
- if (mark.is_neutral()) {
- // Anticipate successful CAS -- the ST of the displaced mark must
- // be visible <= the ST performed by the CAS.
- lock->set_displaced_header(mark);
- if (mark == obj()->cas_set_mark(markWord::from_pointer(lock), mark)) {
- return;
+ // The object header will never be displaced to this lock,
+ // so it does not matter what the value is, except that it
+ // must be non-zero to avoid looking like a re-entrant lock,
+ // and must not look locked either.
+ lock->set_displaced_header(markWord::unused_mark());
}
- // Fall through to inflate() ...
- } else if (mark.has_locker() &&
- current->is_lock_owned((address)mark.locker())) {
- assert(lock != mark.locker(), "must not re-lock the same lock");
- assert(lock != (BasicLock*)obj->mark().value(), "don't relock with same BasicLock");
- lock->set_displaced_header(markWord::from_pointer(NULL));
- return;
}
- // The object header will never be displaced to this lock,
- // so it does not matter what the value is, except that it
- // must be non-zero to avoid looking like a re-entrant lock,
- // and must not look locked either.
- lock->set_displaced_header(markWord::unused_mark());
// An async deflation can race after the inflate() call and before
// enter() can make the ObjectMonitor busy. enter() returns false if
// we have lost the race to async deflation and we simply try again.
while (true) {
ObjectMonitor* monitor = inflate(current, obj(), inflate_cause_monitor_enter);
}
}
}
void ObjectSynchronizer::exit(oop object, BasicLock* lock, JavaThread* current) {
- markWord mark = object->mark();
- // We cannot check for Biased Locking if we are racing an inflation.
- assert(mark == markWord::INFLATING() ||
- !mark.has_bias_pattern(), "should not see bias pattern here");
-
- markWord dhw = lock->displaced_header();
- if (dhw.value() == 0) {
- // If the displaced header is NULL, then this exit matches up with
- // a recursive enter. No real work to do here except for diagnostics.
- #ifndef PRODUCT
- if (mark != markWord::INFLATING()) {
- // Only do diagnostics if we are not racing an inflation. Simply
- // exiting a recursive enter of a Java Monitor that is being
- // inflated is safe; see the has_monitor() comment below.
- assert(!mark.is_neutral(), "invariant");
- assert(!mark.has_locker() ||
- current->is_lock_owned((address)mark.locker()), "invariant");
- if (mark.has_monitor()) {
- // The BasicLock's displaced_header is marked as a recursive
- // enter and we have an inflated Java Monitor (ObjectMonitor).
- // This is a special case where the Java Monitor was inflated
- // after this thread entered the stack-lock recursively. When a
- // Java Monitor is inflated, we cannot safely walk the Java
- // Monitor owner's stack and update the BasicLocks because a
- // Java Monitor can be asynchronously inflated by a thread that
- // does not own the Java Monitor.
- ObjectMonitor* m = mark.monitor();
- assert(m->object()->mark() == mark, "invariant");
- assert(m->is_entered(current), "invariant");
+ if (!useHeavyMonitors()) {
+ markWord mark = object->mark();
+ if (LockingMode == LM_LIGHTWEIGHT) {
+ // Fast-locking does not use the 'lock' argument.
+ if (mark.is_fast_locked()) {
+ markWord unlocked_mark = mark.set_unlocked();
+ markWord old_mark = object->cas_set_mark(unlocked_mark, mark);
+ if (old_mark != mark) {
+ // Another thread won the CAS, it must have inflated the monitor.
+ // It can only have installed an anonymously locked monitor at this point.
+ // Fetch that monitor, set owner correctly to this thread, and
+ // exit it (allowing waiting threads to enter).
+ assert(old_mark.has_monitor(), "must have monitor");
+ ObjectMonitor* monitor = old_mark.monitor();
+ assert(monitor->is_owner_anonymous(), "must be anonymous owner");
+ monitor->set_owner_from_anonymous(current);
+ monitor->exit(current);
+ }
+ LockStack& lock_stack = current->lock_stack();
+ lock_stack.remove(object);
+ return;
}
- }
+ } else if (LockingMode == LM_LEGACY) {
+ markWord dhw = lock->displaced_header();
+ if (dhw.value() == 0) {
+ // If the displaced header is NULL, then this exit matches up with
+ // a recursive enter. No real work to do here except for diagnostics.
+ #ifndef PRODUCT
+ if (mark != markWord::INFLATING()) {
+ // Only do diagnostics if we are not racing an inflation. Simply
+ // exiting a recursive enter of a Java Monitor that is being
+ // inflated is safe; see the has_monitor() comment below.
+ assert(!mark.is_neutral(), "invariant");
+ assert(!mark.has_locker() ||
+ current->is_lock_owned((address)mark.locker()), "invariant");
+ if (mark.has_monitor()) {
+ // The BasicLock's displaced_header is marked as a recursive
+ // enter and we have an inflated Java Monitor (ObjectMonitor).
+ // This is a special case where the Java Monitor was inflated
+ // after this thread entered the stack-lock recursively. When a
+ // Java Monitor is inflated, we cannot safely walk the Java
+ // Monitor owner's stack and update the BasicLocks because a
+ // Java Monitor can be asynchronously inflated by a thread that
+ // does not own the Java Monitor.
+ ObjectMonitor* m = mark.monitor();
+ assert(m->object()->mark() == mark, "invariant");
+ assert(m->is_entered(current), "invariant");
+ }
+ }
#endif
- return;
- }
+ return;
+ }
- if (mark == markWord::from_pointer(lock)) {
- // If the object is stack-locked by the current thread, try to
- // swing the displaced header from the BasicLock back to the mark.
- assert(dhw.is_neutral(), "invariant");
- if (object->cas_set_mark(dhw, mark) == mark) {
- return;
+ if (mark == markWord::from_pointer(lock)) {
+ // If the object is stack-locked by the current thread, try to
+ // swing the displaced header from the BasicLock back to the mark.
+ assert(dhw.is_neutral(), "invariant");
+ if (object->cas_set_mark(dhw, mark) == mark) {
+ return;
+ }
+ }
}
}
// We have to take the slow-path of possible inflation and then exit.
// The ObjectMonitor* can't be async deflated until ownership is
// dropped inside exit() and the ObjectMonitor* must be !is_busy().
ObjectMonitor* monitor = inflate(current, object, inflate_cause_vm_internal);
+ if (LockingMode == LM_LIGHTWEIGHT && monitor->is_owner_anonymous()) {
+ // It must be us. Pop lock object from lock stack.
+ LockStack& lock_stack = current->lock_stack();
+ oop popped = lock_stack.pop();
+ assert(popped == object, "must be owned by this thread");
+ monitor->set_owner_from_anonymous(current);
+ }
monitor->exit(current);
}
// -----------------------------------------------------------------------------
// Class Loader support to workaround deadlocks on the class loader lock objects
BiasedLocking::revoke(current, obj);
assert(!obj->mark().has_bias_pattern(), "biases should be revoked by now");
}
markWord mark = obj->mark();
- if (mark.has_locker() && current->is_lock_owned((address)mark.locker())) {
- // Not inflated so there can't be any waiters to notify.
- return;
+ if (LockingMode == LM_LIGHTWEIGHT) {
+ if ((mark.is_fast_locked() && current->lock_stack().contains(obj()))) {
+ // Not inflated so there can't be any waiters to notify.
+ return;
+ }
+ } else if (LockingMode == LM_LEGACY) {
+ if (mark.has_locker() && current->is_lock_owned((address)mark.locker())) {
+ // Not inflated so there can't be any waiters to notify.
+ return;
+ }
}
// The ObjectMonitor* can't be async deflated until ownership is
// dropped by the calling thread.
ObjectMonitor* monitor = inflate(current, obj(), inflate_cause_notify);
monitor->notify(CHECK);
BiasedLocking::revoke(current, obj);
assert(!obj->mark().has_bias_pattern(), "biases should be revoked by now");
}
markWord mark = obj->mark();
- if (mark.has_locker() && current->is_lock_owned((address)mark.locker())) {
- // Not inflated so there can't be any waiters to notify.
- return;
+ if (LockingMode == LM_LIGHTWEIGHT) {
+ if ((mark.is_fast_locked() && current->lock_stack().contains(obj()))) {
+ // Not inflated so there can't be any waiters to notify.
+ return;
+ }
+ } else if (LockingMode == LM_LEGACY) {
+ if (mark.has_locker() && current->is_lock_owned((address)mark.locker())) {
+ // Not inflated so there can't be any waiters to notify.
+ return;
+ }
}
// The ObjectMonitor* can't be async deflated until ownership is
// dropped by the calling thread.
ObjectMonitor* monitor = inflate(current, obj(), inflate_cause_notify);
monitor->notifyAll(CHECK);
static SharedGlobals GVars;
static markWord read_stable_mark(oop obj) {
markWord mark = obj->mark_acquire();
- if (!mark.is_being_inflated()) {
+ if (!mark.is_being_inflated() || LockingMode == LM_LIGHTWEIGHT) {
+ // New lightweight locking does not use the markWord::INFLATING() protocol.
return mark; // normal fast-path return
}
int its = 0;
for (;;) {
v = (v ^ (v >> 19)) ^ (t ^ (t >> 8));
current->_hashStateW = v;
value = v;
}
- value &= markWord::hash_mask;
+ value &= UseCompactObjectHeaders ? markWord::hash_mask_compact : markWord::hash_mask;
if (value == 0) value = 0xBAD;
assert(value != markWord::no_hash, "invariant");
return value;
}
+ // Can be called from non JavaThreads (e.g., VMThread) for FastHashCode
+ // calculations as part of JVM/TI tagging.
+ static bool is_lock_owned(Thread* thread, oop obj) {
+ assert(LockingMode == LM_LIGHTWEIGHT, "only call this with new lightweight locking enabled");
+ return thread->is_Java_thread() ? reinterpret_cast<JavaThread*>(thread)->lock_stack().contains(obj) : false;
+ }
+
intptr_t ObjectSynchronizer::FastHashCode(Thread* current, oop obj) {
if (UseBiasedLocking) {
// NOTE: many places throughout the JVM do not expect a safepoint
// to be taken here. However, we only ever bias Java instances and all
// of the call sites of identity_hash that might revoke biases have
}
return hash;
}
// Fall thru so we only have one place that installs the hash in
// the ObjectMonitor.
- } else if (current->is_lock_owned((address)mark.locker())) {
+ } else if (LockingMode == LM_LIGHTWEIGHT && mark.is_fast_locked() && is_lock_owned(current, obj)) {
+ // This is a fast lock owned by the calling thread so use the
+ // markWord from the object.
+ hash = mark.hash();
+ if (hash != 0) { // if it has a hash, just return it
+ return hash;
+ }
+ } else if (LockingMode == LM_LEGACY && mark.has_locker() && current->is_lock_owned((address)mark.locker())) {
// This is a stack lock owned by the calling thread so fetch the
// displaced markWord from the BasicLock on the stack.
temp = mark.displaced_mark_helper();
assert(temp.is_neutral(), "invariant: header=" INTPTR_FORMAT, temp.value());
hash = temp.hash();
assert(current == JavaThread::current(), "Can only be called on current thread");
oop obj = h_obj();
markWord mark = read_stable_mark(obj);
- // Uncontended case, header points to stack
- if (mark.has_locker()) {
+ if (LockingMode == LM_LEGACY && mark.has_locker()) {
+ // stack-locked case, header points into owner's stack
return current->is_lock_owned((address)mark.locker());
}
+
+ if (LockingMode == LM_LIGHTWEIGHT && mark.is_fast_locked()) {
+ // fast-locking case, see if lock is in current's lock stack
+ return current->lock_stack().contains(h_obj());
+ }
+
// Contended case, header points to ObjectMonitor (tagged pointer)
if (mark.has_monitor()) {
// The first stage of async deflation does not affect any field
// used by this comparison so the ObjectMonitor* is usable here.
ObjectMonitor* monitor = mark.monitor();
}
assert(!h_obj->mark().has_bias_pattern(), "biases should be revoked by now");
}
oop obj = h_obj();
- address owner = NULL;
-
markWord mark = read_stable_mark(obj);
- // Uncontended case, header points to stack
- if (mark.has_locker()) {
- owner = (address) mark.locker();
+ if (LockingMode == LM_LEGACY && mark.has_locker()) {
+ // stack-locked so header points into owner's stack.
+ // owning_thread_from_monitor_owner() may also return null here:
+ return Threads::owning_thread_from_monitor_owner(t_list, (address) mark.locker());
+ }
+
+ if (LockingMode == LM_LIGHTWEIGHT && mark.is_fast_locked()) {
+ // fast-locked so get owner from the object.
+ // owning_thread_from_object() may also return null here:
+ return Threads::owning_thread_from_object(t_list, h_obj());
}
// Contended case, header points to ObjectMonitor (tagged pointer)
- else if (mark.has_monitor()) {
+ if (mark.has_monitor()) {
// The first stage of async deflation does not affect any field
// used by this comparison so the ObjectMonitor* is usable here.
ObjectMonitor* monitor = mark.monitor();
assert(monitor != NULL, "monitor should be non-null");
- owner = (address) monitor->owner();
- }
-
- if (owner != NULL) {
- // owning_thread_from_monitor_owner() may also return NULL here
- return Threads::owning_thread_from_monitor_owner(t_list, owner);
+ // owning_thread_from_monitor() may also return null here:
+ return Threads::owning_thread_from_monitor(t_list, monitor);
}
// Unlocked case, header in place
// Cannot have assertion since this object may have been
// locked by another thread when reaching here.
for (;;) {
const markWord mark = object->mark_acquire();
assert(!mark.has_bias_pattern(), "invariant");
// The mark can be in one of the following states:
- // * Inflated - just return
- // * Stack-locked - coerce it to inflated
+ // * inflated - Just return if using stack-locking.
+ // If using fast-locking and the ObjectMonitor owner
+ // is anonymous and the current thread owns the
+ // object lock, then we make the current thread the
+ // ObjectMonitor owner and remove the lock from the
+ // current thread's lock stack.
+ // * fast-locked - Coerce it to inflated from fast-locked.
+ // * stack-locked - Coerce it to inflated from stack-locked.
// * INFLATING - busy wait for conversion to complete
// * Neutral - aggressively inflate the object.
// * BIASED - Illegal. We should never see this
// CASE: inflated
if (mark.has_monitor()) {
ObjectMonitor* inf = mark.monitor();
markWord dmw = inf->header();
assert(dmw.is_neutral(), "invariant: header=" INTPTR_FORMAT, dmw.value());
+ if (LockingMode == LM_LIGHTWEIGHT && inf->is_owner_anonymous() && is_lock_owned(current, object)) {
+ inf->set_owner_from_anonymous(current);
+ assert(current->is_Java_thread(), "must be Java thread");
+ reinterpret_cast<JavaThread*>(current)->lock_stack().remove(object);
+ }
return inf;
}
// CASE: inflation in progress - inflating over a stack-lock.
// Some other thread is converting from stack-locked to inflated.
// Only that thread can complete inflation -- other threads must wait.
// The INFLATING value is transient.
// Currently, we spin/yield/park and poll the markword, waiting for inflation to finish.
// We could always eliminate polling by parking the thread on some auxiliary list.
- if (mark == markWord::INFLATING()) {
- read_stable_mark(object);
- continue;
+ if (LockingMode != LM_LIGHTWEIGHT) {
+ // New lightweight locking does not use INFLATING.
+ // CASE: inflation in progress - inflating over a stack-lock.
+ // Some other thread is converting from stack-locked to inflated.
+ // Only that thread can complete inflation -- other threads must wait.
+ // The INFLATING value is transient.
+ // Currently, we spin/yield/park and poll the markword, waiting for inflation to finish.
+ // We could always eliminate polling by parking the thread on some auxiliary list.
+ if (mark == markWord::INFLATING()) {
+ read_stable_mark(object);
+ continue;
+ }
+ }
+
+ // CASE: fast-locked
+ // Could be fast-locked either by current or by some other thread.
+ //
+ // Note that we allocate the ObjectMonitor speculatively, _before_
+ // attempting to set the object's mark to the new ObjectMonitor. If
+ // this thread owns the monitor, then we set the ObjectMonitor's
+ // owner to this thread. Otherwise, we set the ObjectMonitor's owner
+ // to anonymous. If we lose the race to set the object's mark to the
+ // new ObjectMonitor, then we just delete it and loop around again.
+ //
+ LogStreamHandle(Trace, monitorinflation) lsh;
+ if (LockingMode == LM_LIGHTWEIGHT && mark.is_fast_locked()) {
+ ObjectMonitor* monitor = new ObjectMonitor(object);
+ monitor->set_header(mark.set_unlocked());
+ bool own = is_lock_owned(current, object);
+ if (own) {
+ // Owned by us.
+ monitor->set_owner_from(NULL, current);
+ } else {
+ // Owned by somebody else.
+ monitor->set_owner_anonymous();
+ }
+ markWord monitor_mark = markWord::encode(monitor);
+ markWord old_mark = object->cas_set_mark(monitor_mark, mark);
+ if (old_mark == mark) {
+ // Success! Return inflated monitor.
+ if (own) {
+ assert(current->is_Java_thread(), "must be Java thread");
+ reinterpret_cast<JavaThread*>(current)->lock_stack().remove(object);
+ }
+ // Once the ObjectMonitor is configured and object is associated
+ // with the ObjectMonitor, it is safe to allow async deflation:
+ _in_use_list.add(monitor);
+
+ // Hopefully the performance counters are allocated on distinct
+ // cache lines to avoid false sharing on MP systems ...
+ OM_PERFDATA_OP(Inflations, inc());
+ if (log_is_enabled(Trace, monitorinflation)) {
+ ResourceMark rm(current);
+ lsh.print_cr("inflate(has_locker): object=" INTPTR_FORMAT ", mark="
+ INTPTR_FORMAT ", type='%s'", p2i(object),
+ object->mark().value(), object->klass()->external_name());
+ }
+ if (event.should_commit()) {
+ post_monitor_inflate_event(&event, object, cause);
+ }
+ return monitor;
+ } else {
+ delete monitor;
+ continue; // Interference -- just retry
+ }
}
// CASE: stack-locked
// Could be stack-locked either by this thread or by some other thread.
//
// allocated the ObjectMonitor, and then finally STed the address of the
// ObjectMonitor into the mark. This was correct, but artificially lengthened
// the interval in which INFLATING appeared in the mark, thus increasing
// the odds of inflation contention.
- LogStreamHandle(Trace, monitorinflation) lsh;
-
- if (mark.has_locker()) {
+ if (LockingMode == LM_LEGACY && mark.has_locker()) {
+ assert(LockingMode != LM_LIGHTWEIGHT, "cannot happen with new lightweight locking");
ObjectMonitor* m = new ObjectMonitor(object);
// Optimistically prepare the ObjectMonitor - anticipate successful CAS
// We do this before the CAS in order to minimize the length of time
// in which INFLATING appears in the mark.
log_trace(monitorinflation)("HandshakeForDeflation::do_thread: thread="
INTPTR_FORMAT, p2i(thread));
}
};
+ class VM_RendezvousGCThreads : public VM_Operation {
+ public:
+ bool evaluate_at_safepoint() const override { return false; }
+ VMOp_Type type() const override { return VMOp_RendezvousGCThreads; }
+ void doit() override {
+ SuspendibleThreadSet::synchronize();
+ SuspendibleThreadSet::desynchronize();
+ };
+ };
+
// This function is called by the MonitorDeflationThread to deflate
// ObjectMonitors. It is also called via do_final_audit_and_print_stats()
// by the VMThread.
size_t ObjectSynchronizer::deflate_idle_monitors() {
Thread* current = Thread::current();
_in_use_list.count(), _in_use_list.max());
}
// A JavaThread needs to handshake in order to safely free the
// ObjectMonitors that were deflated in this cycle.
+ // Also, we sync and desync GC threads around the handshake, so that they can
+ // safely read the mark-word and look-through to the object-monitor, without
+ // being afraid that the object-monitor is going away.
HandshakeForDeflation hfd_hc;
Handshake::execute(&hfd_hc);
+ VM_RendezvousGCThreads sync_gc;
+ VMThread::execute(&sync_gc);
if (ls != NULL) {
ls->print_cr("after handshaking: in_use_list stats: ceiling="
SIZE_FORMAT ", count=" SIZE_FORMAT ", max=" SIZE_FORMAT,
in_use_list_ceiling(), _in_use_list.count(), _in_use_list.max());
< prev index next >