< prev index next >

src/hotspot/share/runtime/synchronizer.cpp

Print this page
@@ -1,7 +1,7 @@
  /*
-  * Copyright (c) 1998, 2021, Oracle and/or its affiliates. All rights reserved.
+  * Copyright (c) 1998, 2024, Oracle and/or its affiliates. All rights reserved.
   * DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER.
   *
   * This code is free software; you can redistribute it and/or modify it
   * under the terms of the GNU General Public License version 2 only, as
   * published by the Free Software Foundation.

@@ -23,23 +23,26 @@
   */
  
  #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 "memory/universe.hpp"
  #include "oops/markWord.hpp"
  #include "oops/oop.inline.hpp"
  #include "runtime/atomic.hpp"
  #include "runtime/biasedLocking.hpp"
+ #include "runtime/globals.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"

@@ -55,10 +58,11 @@
  #include "runtime/vframe.hpp"
  #include "runtime/vmThread.hpp"
  #include "utilities/align.hpp"
  #include "utilities/dtrace.hpp"
  #include "utilities/events.hpp"
+ #include "utilities/globalDefinitions.hpp"
  #include "utilities/preserveException.hpp"
  
  void MonitorList::add(ObjectMonitor* m) {
    ObjectMonitor* head;
    do {

@@ -273,14 +277,22 @@
    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");

@@ -324,10 +336,22 @@
  
    if (obj->klass()->is_value_based()) {
      return false;
    }
  
+   if (LockingMode == LM_LIGHTWEIGHT) {
+     LockStack& lock_stack = current->lock_stack();
+     if (lock_stack.is_full()) {
+       // Always go into runtime if the lock stack is full.
+       return false;
+     }
+     if (lock_stack.try_recursive_enter(obj)) {
+       // Recursive lock successful.
+       return true;
+     }
+   }
+ 
    const markWord mark = obj->mark();
  
    if (mark.has_monitor()) {
      ObjectMonitor* const m = mark.monitor();
      // An async deflation or GC can race us before we manage to make

@@ -346,21 +370,23 @@
      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;
      }

@@ -375,12 +401,13 @@
  
    return false;        // revert to slow-path
  }
  
  // Handle notifications when synchronizing on value based classes
- void ObjectSynchronizer::handle_sync_on_value_based_class(Handle obj, JavaThread* current) {
-   frame last_frame = current->last_frame();
+ void ObjectSynchronizer::handle_sync_on_value_based_class(Handle obj, JavaThread* locking_thread) {
+   assert(locking_thread == Thread::current() || locking_thread->is_obj_deopt_suspend(), "must be");
+   frame last_frame = locking_thread->last_frame();
    bool bcp_was_adjusted = false;
    // Don't decrement bcp if it points to the frame's first instruction.  This happens when
    // handle_sync_on_value_based_class() is called because of a synchronized method.  There
    // is no actual monitorenter instruction in the byte code in this case.
    if (last_frame.is_interpreted_frame() &&

@@ -389,28 +416,28 @@
      last_frame.interpreter_frame_set_bcp(last_frame.interpreter_frame_bcp() - 1);
      bcp_was_adjusted = true;
    }
  
    if (DiagnoseSyncOnValueBasedClasses == FATAL_EXIT) {
-     ResourceMark rm(current);
+     ResourceMark rm;
      stringStream ss;
-     current->print_stack_on(&ss);
+     locking_thread->print_stack_on(&ss);
      char* base = (char*)strstr(ss.base(), "at");
      char* newline = (char*)strchr(ss.base(), '\n');
      if (newline != NULL) {
        *newline = '\0';
      }
      fatal("Synchronizing on object " INTPTR_FORMAT " of klass %s %s", p2i(obj()), obj->klass()->external_name(), base);
    } else {
      assert(DiagnoseSyncOnValueBasedClasses == LOG_WARNING, "invalid value for DiagnoseSyncOnValueBasedClasses");
-     ResourceMark rm(current);
+     ResourceMark rm;
      Log(valuebasedclasses) vblog;
  
      vblog.info("Synchronizing on object " INTPTR_FORMAT " of klass %s", p2i(obj()), obj->klass()->external_name());
-     if (current->has_last_Java_frame()) {
+     if (locking_thread->has_last_Java_frame()) {
        LogStream info_stream(vblog.info());
-       current->print_stack_on(&info_stream);
+       locking_thread->print_stack_on(&info_stream);
      } else {
        vblog.info("Cannot find the last Java frame");
      }
  
      EventSyncOnValueBasedClass event;

@@ -423,109 +450,233 @@
    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
+ 
+ void ObjectSynchronizer::enter_for(Handle obj, BasicLock* lock, JavaThread* locking_thread) {
+   // When called with locking_thread != Thread::current() some mechanism must synchronize
+   // the locking_thread with respect to the current thread. Currently only used when
+   // deoptimizing and re-locking locks. See Deoptimization::relock_objects
+   assert(locking_thread == Thread::current() || locking_thread->is_obj_deopt_suspend(), "must be");
+   if (!enter_fast_impl(obj, lock, locking_thread)) {
+     // Inflated ObjectMonitor::enter_for is required
+ 
+     // An async deflation can race after the inflate_for() call and before
+     // enter_for() can make the ObjectMonitor busy. enter_for() returns false
+     // if we have lost the race to async deflation and we simply try again.
+     while (true) {
+       ObjectMonitor* monitor = inflate_for(locking_thread, obj(), inflate_cause_monitor_enter);
+       if (monitor->enter_for(locking_thread)) {
+         return;
+       }
+       assert(monitor->is_being_async_deflated(), "must be");
+     }
+   }
+ }
+ 
+ void ObjectSynchronizer::enter(Handle obj, BasicLock* lock, JavaThread* current) {
+   assert(current == Thread::current(), "must be");
+   if (!enter_fast_impl(obj, lock, current)) {
+     // Inflated ObjectMonitor::enter is required
+ 
+     // 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);
+       if (monitor->enter(current)) {
+         return;
+       }
+     }
+   }
+ }
+ 
  // 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.
+ bool ObjectSynchronizer::enter_fast_impl(Handle obj, BasicLock* lock, JavaThread* locking_thread) {
  
- void ObjectSynchronizer::enter(Handle obj, BasicLock* lock, JavaThread* current) {
    if (obj->klass()->is_value_based()) {
-     handle_sync_on_value_based_class(obj, current);
-   }
+     handle_sync_on_value_based_class(obj, locking_thread);
+   }
+ 
+   if (!useHeavyMonitors()) {
+     if (LockingMode == LM_LIGHTWEIGHT) {
+       // Fast-locking does not use the 'lock' argument.
+       LockStack& lock_stack = locking_thread->lock_stack();
+       if (lock_stack.is_full()) {
+         // We unconditionally make room on the lock stack by inflating
+         // the least recently locked object on the lock stack.
+ 
+         // About the choice to inflate least recently locked object.
+         // First we must chose to inflate a lock, either some lock on
+         // the lock-stack or the lock that is currently being entered
+         // (which may or may not be on the lock-stack).
+         // Second the best lock to inflate is a lock which is entered
+         // in a control flow where there are only a very few locks being
+         // used, as the costly part of inflated locking is inflation,
+         // not locking. But this property is entirely program dependent.
+         // Third inflating the lock currently being entered on when it
+         // is not present on the lock-stack will result in a still full
+         // lock-stack. This creates a scenario where every deeper nested
+         // monitorenter must call into the runtime.
+         // The rational here is as follows:
+         // Because we cannot (currently) figure out the second, and want
+         // to avoid the third, we inflate a lock on the lock-stack.
+         // The least recently locked lock is chosen as it is the lock
+         // with the longest critical section.
+ 
+         log_info(monitorinflation)("LockStack capacity exceeded, inflating.");
+         ObjectMonitor* monitor = inflate_for(locking_thread, lock_stack.bottom(), inflate_cause_vm_internal);
+         assert(monitor->owner() == Thread::current(), "must be owner=" PTR_FORMAT " current=" PTR_FORMAT " mark=" PTR_FORMAT,
+                p2i(monitor->owner()), p2i(Thread::current()), monitor->object()->mark_acquire().value());
+         assert(!lock_stack.is_full(), "must have made room here");
+       }
  
-   if (UseBiasedLocking) {
-     BiasedLocking::revoke(current, obj);
-   }
+       markWord mark = obj()->mark_acquire();
+       while (mark.is_neutral()) {
+         // Retry until a lock state change has been observed. cas_set_mark() may collide with non lock bits modifications.
+         // Try to swing into 'fast-locked' state.
+         assert(!lock_stack.contains(obj()), "thread must not already hold the lock");
+         const markWord locked_mark = mark.set_fast_locked();
+         const 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 true;
+         }
+         mark = old_mark;
+       }
  
-   markWord mark = obj->mark();
-   assert(!mark.has_bias_pattern(), "should not see bias pattern here");
+       if (mark.is_fast_locked() && lock_stack.try_recursive_enter(obj())) {
+         // Recursive lock successful.
+         return true;
+       }
  
-   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;
-   }
+       // Failed to fast lock.
+       return false;
+     } else if (LockingMode == LM_LEGACY) {
+       if (UseBiasedLocking) {
+         BiasedLocking::revoke(locking_thread, obj);
+       }
  
-   // 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);
-     if (monitor->enter(current)) {
-       return;
+       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 true;
+         }
+       } else if (mark.has_locker() &&
+                  locking_thread->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 true;
+       }
+ 
+       // 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());
+ 
+       // Failed to fast lock.
+       return false;
      }
    }
+ 
+   return false;
  }
  
  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.
+       LockStack& lock_stack = current->lock_stack();
+       if (mark.is_fast_locked() && lock_stack.try_recursive_exit(object)) {
+         // Recursively unlocked.
+         return;
        }
-     }
+ 
+       if (mark.is_fast_locked() && lock_stack.is_recursive(object)) {
+         // This lock is recursive but is not at the top of the lock stack so we're
+         // doing an unbalanced exit. We have to fall thru to inflation below and
+         // let ObjectMonitor::exit() do the unlock.
+       } else {
+         while (mark.is_fast_locked()) {
+           // Retry until a lock state change has been observed. cas_set_mark() may collide with non lock bits modifications.
+           const markWord unlocked_mark = mark.set_unlocked();
+           const markWord old_mark = object->cas_set_mark(unlocked_mark, mark);
+           if (old_mark == mark) {
+             size_t recursions = lock_stack.remove(object) - 1;
+             assert(recursions == 0, "must not be recursive here");
+             return;
+           }
+           mark = old_mark;
+         }
+       }
+     } 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);
+   assert(!monitor->is_owner_anonymous(), "must not be");
    monitor->exit(current);
  }
  
  // -----------------------------------------------------------------------------
  // Class Loader  support to workaround deadlocks on the class loader lock objects

@@ -686,13 +837,20 @@
      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);

@@ -705,13 +863,20 @@
      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);

@@ -733,11 +898,12 @@
  
  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 (;;) {

@@ -842,11 +1008,11 @@
      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;
  }
  

@@ -878,11 +1044,11 @@
      markWord mark = read_stable_mark(obj);
  
      // object should remain ineligible for biased locking
      assert(!mark.has_bias_pattern(), "invariant");
  
-     if (mark.is_neutral()) {               // if this is a normal header
+     if (mark.is_neutral() || (LockingMode == LM_LIGHTWEIGHT && mark.is_fast_locked())) {
        hash = mark.hash();
        if (hash != 0) {                     // if it has a hash, just return it
          return hash;
        }
        hash = get_next_hash(current, obj);  // get a new hash

@@ -890,10 +1056,14 @@
                                             // try to install the hash
        test = obj->cas_set_mark(temp, mark);
        if (test == mark) {                  // if the hash was installed, return it
          return hash;
        }
+       if (LockingMode == LM_LIGHTWEIGHT) {
+         // CAS failed, retry
+         continue;
+       }
        // Failed to install the hash. It could be that another thread
        // installed the hash just before our attempt or inflation has
        // occurred or... so we fall thru to inflate the monitor for
        // stability and then install the hash.
      } else if (mark.has_monitor()) {

@@ -921,11 +1091,11 @@
          }
          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_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();

@@ -998,14 +1168,20 @@
    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();

@@ -1026,31 +1202,32 @@
      }
      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.

@@ -1240,42 +1417,136 @@
      return;
    }
    (void)inflate(Thread::current(), obj, inflate_cause_vm_internal);
  }
  
- ObjectMonitor* ObjectSynchronizer::inflate(Thread* current, oop object,
-                                            const InflateCause cause) {
+ ObjectMonitor* ObjectSynchronizer::inflate(Thread* current, oop obj, const InflateCause cause) {
+   assert(current == Thread::current(), "must be");
+   if (LockingMode == LM_LIGHTWEIGHT && current->is_Java_thread()) {
+     return inflate_impl(current->as_Java_thread(), obj, cause);
+   }
+   return inflate_impl(nullptr, obj, cause);
+ }
+ 
+ ObjectMonitor* ObjectSynchronizer::inflate_for(JavaThread* thread, oop obj, const InflateCause cause) {
+   assert(thread == Thread::current() || thread->is_obj_deopt_suspend(), "must be");
+   return inflate_impl(thread, obj, cause);
+ }
+ 
+ ObjectMonitor* ObjectSynchronizer::inflate_impl(JavaThread* inflating_thread, oop object, const InflateCause cause) {
+   // The JavaThread* inflating_thread parameter is only used by LM_LIGHTWEIGHT and requires
+   // that the inflating_thread == Thread::current() or is suspended throughout the call by
+   // some other mechanism.
+   // Even with LM_LIGHTWEIGHT the thread might be nullptr when called from a non
+   // JavaThread. (As may still be the case from FastHashCode). However it is only
+   // important for the correctness of the LM_LIGHTWEIGHT algorithm that the thread
+   // is set when called from ObjectSynchronizer::enter from the owning thread,
+   // ObjectSynchronizer::enter_for from any thread, or ObjectSynchronizer::exit.
    EventJavaMonitorInflate event;
  
    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 inflating_thread owns the
+     //                   object lock, then we make the inflating_thread
+     //                   the ObjectMonitor owner and remove the lock from
+     //                   the inflating_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() &&
+           inflating_thread != nullptr && inflating_thread->lock_stack().contains(object)) {
+         inf->set_owner_from_anonymous(inflating_thread);
+         size_t removed = inflating_thread->lock_stack().remove(object);
+         inf->set_recursions(removed - 1);
+       }
        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 the inflating_thread 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
+     // the inflating_thread owns the monitor, then we set the ObjectMonitor's
+     // owner to the inflating_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 = inflating_thread != nullptr && inflating_thread->lock_stack().contains(object);
+       if (own) {
+         // Owned by us.
+         monitor->set_owner_from(NULL, inflating_thread);
+       } 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) {
+           size_t removed = inflating_thread->lock_stack().remove(object);
+           monitor->set_recursions(removed - 1);
+         }
+         // 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;
+           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.
      //

@@ -1284,13 +1555,12 @@
      // 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.
  

@@ -1358,11 +1628,11 @@
  
        // 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);
+         ResourceMark rm;
          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()) {

@@ -1402,11 +1672,11 @@
  
      // 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);
+       ResourceMark rm;
        lsh.print_cr("inflate(neutral): object=" INTPTR_FORMAT ", mark="
                     INTPTR_FORMAT ", type='%s'", p2i(object),
                     object->mark().value(), object->klass()->external_name());
      }
      if (event.should_commit()) {

@@ -1479,10 +1749,20 @@
      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();

@@ -1531,12 +1811,17 @@
                       _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 >