< prev index next >

src/java.base/share/classes/java/lang/ref/Reference.java

Print this page

        

@@ -23,15 +23,16 @@
  * questions.
  */
 
 package java.lang.ref;
 
-import sun.misc.Cleaner;
 import sun.misc.JavaLangRefAccess;
 import sun.misc.ManagedLocalsThread;
 import sun.misc.SharedSecrets;
 
+import java.util.concurrent.ThreadLocalRandom;
+
 /**
  * Abstract base class for reference objects.  This class defines the
  * operations common to all reference objects.  Because reference objects are
  * implemented in close cooperation with the garbage collector, this class may
  * not be subclassed directly.

@@ -97,38 +98,52 @@
     /* When active:   NULL
      *     pending:   this
      *    Enqueued:   next reference in queue (or this if last)
      *    Inactive:   this
      */
-    @SuppressWarnings("rawtypes")
-    Reference next;
+    Reference<?> next;
 
     /* When active:   next element in a discovered reference list maintained by GC (or this if last)
      *     pending:   next element in the pending list (or null if last)
      *   otherwise:   NULL
      */
-    transient private Reference<T> discovered;  /* used by VM */
+    transient private Reference<?> discovered;  /* used by VM */
 
 
     /* Object used to synchronize with the garbage collector.  The collector
      * must acquire this lock at the beginning of each collection cycle.  It is
      * therefore critical that any code holding this lock complete as quickly
      * as possible, allocate no new objects, and avoid calling user code.
      */
     static private class Lock { }
-    private static Lock lock = new Lock();
+    private static final Lock lock = new Lock();
 
 
     /* List of References waiting to be enqueued.  The collector adds
      * References to this list, while the Reference-handler thread removes
      * them.  This list is protected by the above lock object. The
      * list uses the discovered field to link its elements.
      */
-    private static Reference<Object> pending = null;
+    private static Reference<?> pending;
+
+    /**
+     * Max. number of Reference(s) to unhook from pending chain in one chunk
+     * before releasing the lock, handling them and grabbing the
+     * lock again.
+     */
+    private static final int UNHOOK_CHUNK_SIZE = 32768;
 
-    /* High-priority thread to enqueue pending References
+    /**
+     * Max. number of j.l.r.Cleaner(s) to execute in one ForkJoinTask
+     */
+    private static final int CLEANER_CHUNK_SIZE = 256;
+
+    /**
+     * Max. number of Reference(s) to enqueue in one chunk
      */
+    private static final int ENQUEUE_CHUNK_SIZE = 256;
+
     private static class ReferenceHandler extends ManagedLocalsThread {
 
         private static void ensureClassInitialized(Class<?> clazz) {
             try {
                 Class.forName(clazz.getName(), true, clazz.getClassLoader());

@@ -140,107 +155,240 @@
         static {
             // pre-load and initialize InterruptedException and Cleaner classes
             // so that we don't get into trouble later in the run loop if there's
             // memory shortage while loading/initializing them lazily.
             ensureClassInitialized(InterruptedException.class);
+            ensureClassInitialized(sun.misc.Cleaner.class);
             ensureClassInitialized(Cleaner.class);
         }
 
         ReferenceHandler(ThreadGroup g, String name) {
             super(g, name);
         }
 
         public void run() {
+            // wait for VM to boot-up before starting reference handling
+            // ForkJoinPool since it needs access to some system properties
+            // and Finalizer needs access to SharedSecrets.
             while (true) {
-                tryHandlePending(true);
+                try {
+                    sun.misc.VM.awaitBooted();
+                    break;
+                } catch (InterruptedException e) {
+                    // ignore;
+                }
+            }
+            // enter endless loop
+            boolean[] morePending = new boolean[1];
+            while (true) {
+                Reference<?> chunk = null;
+                try {
+                    synchronized (lock) {
+                        chunk = Reference.unhookPendingChunk(UNHOOK_CHUNK_SIZE, morePending);
+                        if (chunk == null) {
+                            // waiting on notification can throw InterruptedException
+                            // if the thread is interrupted, but also OutOfMemoryError
+                            // if the InterruptedException can not be allocated.
+                            lock.wait();
+                            // since we have already re-obtained the lock, we can
+                            // re-try poll and will typically get a non-null chunk.
+                            chunk = Reference.unhookPendingChunk(UNHOOK_CHUNK_SIZE, morePending);
+                        }
+                    }
+                } catch (OutOfMemoryError e) {
+                    // give other threads some time so they hopefully release some
+                    // references and GC reclaims some space, then retry...
+                    Thread.yield();
+                } catch (InterruptedException e) {
+                    // ignore
+                }
+                if (chunk != null) {
+                    if (morePending[0]) {
+                        // submit a handling task and return for next chunk
+                        new ReferenceHandling.PendingChunkHandler(chunk).submit();
+                    } else {
+                        // no more pending, so we can handle the chunk directly
+                        Reference.handlePendingChunk(chunk);
+                    }
+                }
             }
         }
     }
 
     /**
-     * Try handle pending {@link Reference} if there is one.<p>
-     * Return {@code true} as a hint that there might be another
-     * {@link Reference} pending or {@code false} when there are no more pending
+     * Try handle a chunk of pending {@link Reference}s if there are any.<p>
+     * Return {@code true} as a hint that there are more
+     * {@link Reference}s pending or {@code false} when there are no more pending
      * {@link Reference}s at the moment and the program can do some other
      * useful work instead of looping.
      *
-     * @param waitForNotify if {@code true} and there was no pending
-     *                      {@link Reference}, wait until notified from VM
-     *                      or interrupted; if {@code false}, return immediately
-     *                      when there is no pending {@link Reference}.
-     * @return {@code true} if there was a {@link Reference} pending and it
-     *         was processed, or we waited for notification and either got it
-     *         or thread was interrupted before being notified;
-     *         {@code false} otherwise.
-     */
-    static boolean tryHandlePending(boolean waitForNotify) {
-        Reference<Object> r;
-        Cleaner c;
-        try {
+     * @return {@code true} if there is be more {@link Reference}s pending.
+     */
+    static boolean tryHandlePending() {
+        Reference<?> r;
             synchronized (lock) {
-                if (pending != null) {
-                    r = pending;
-                    // 'instanceof' might throw OutOfMemoryError sometimes
-                    // so do this before un-linking 'r' from the 'pending' chain...
-                    c = r instanceof Cleaner ? (Cleaner) r : null;
-                    // unlink 'r' from 'pending' chain
-                    pending = r.discovered;
-                    r.discovered = null;
-                } else {
-                    // The waiting on the lock may cause an OutOfMemoryError
-                    // because it may try to allocate exception objects.
-                    if (waitForNotify) {
-                        lock.wait();
+            r = unhookPendingChunk(UNHOOK_CHUNK_SIZE, null);
                     }
-                    // retry if waited
-                    return waitForNotify;
+        if (r == null) return false;
+        handlePendingChunk(r);
+        synchronized (lock) {
+            return pending != null;
                 }
             }
-        } catch (OutOfMemoryError x) {
-            // Give other threads CPU time so they hopefully drop some live references
-            // and GC reclaims some space.
-            // Also prevent CPU intensive spinning in case 'r instanceof Cleaner' above
-            // persistently throws OOME for some time...
-            Thread.yield();
-            // retry
-            return true;
-        } catch (InterruptedException x) {
-            // retry
-            return true;
-        }
-
-        // Fast path for cleaners
-        if (c != null) {
-            c.clean();
-            return true;
-        }
-
-        ReferenceQueue<? super Object> q = r.queue;
-        if (q != ReferenceQueue.NULL) q.enqueue(r);
-        return true;
+
+    /**
+     * Unhooks a chunk of max. {@code chunkSize} references from pending chain and
+     * returns the head of the chunk; elements of the chunk can be reached using
+     * {@link #discovered} links; the last in chunk is linked to null.
+     *
+     * @param chunkSize max. number of references to unhook from the pending chain
+     * @param morePending   if non null, it should be a boolean array with length 1
+     *                      to hold the additional result - a flag indicating that
+     *                      there are more pending references waiting after a chunk
+     *                      of them has been returned.
+     * @return the head of the chunk of max. {@code chunkSize} pending references or
+     * null if there are none pending.
+     */
+    private static Reference<?> unhookPendingChunk(int chunkSize, boolean[] morePending) {
+        // assert Thread.holdsLock(lock);
+        Reference<?> r;
+        if ((r = pending) != null) {
+            // skip over max. chunkSize references using 'discovered' link
+            Reference<?> p = r;
+            Reference<?> d = p.discovered;
+            for (int i = 0;
+                 d != null && ++i < chunkSize;
+                 p = d, d = p.discovered) {}
+            pending = d;
+            p.discovered = null; // unlink last in unhooked chunk from the rest
+            if (morePending != null) morePending[0] = (d != null);
+        } else {
+            if (morePending != null) morePending[0] = false;
+        }
+        return r;
     }
 
-    static {
-        ThreadGroup tg = Thread.currentThread().getThreadGroup();
-        for (ThreadGroup tgn = tg;
-             tgn != null;
-             tg = tgn, tgn = tg.getParent());
-        Thread handler = new ReferenceHandler(tg, "Reference Handler");
-        /* If there were a special system-only priority greater than
-         * MAX_PRIORITY, it would be used here
-         */
-        handler.setPriority(Thread.MAX_PRIORITY);
-        handler.setDaemon(true);
-        handler.start();
+    /**
+     * Takes a non-null chunk of unhooked pending references
+     * (obtained using {@link #unhookPendingChunk}) and handles
+     * them as following:
+     * <ul>
+     * <li>sun.misc.Cleaner(s) are executed immediately</li>
+     * <li>java.lang.ref.Cleaner(s) are submitted in chunks as ForkJoinTask(s)</li>
+     * <li>all other Reference(s) are enqueued in their respective queues</li>
+     * </ul>
+     * The references in chunk are linked using {@link #discovered} link.
+     * Last in chunk is linked to null. As they are handled, their discovered
+     * links are reset to null.
+     *
+     * @param chunk the head of a chunk of pending references
+     */
+    static void handlePendingChunk(Reference<?> chunk) {
+        // batch j.l.r.Cleaner(s)
+        Reference<?> cleaners = null;
+        int cleanersCount = 0;
+        // batch runs of consecutive references having same queue
+        Reference<?> referencesHead = null, referencesTail = null;
+        int referencesCount = 0;
+        ReferenceQueue<?> referenceQueue = null;
+        // dispatch references to appropriate targets
+        for (Reference<?> r = chunk, d = r.discovered;
+             r != null;
+             r = d, d = (r == null) ? null : r.discovered) {
+            // invariant established by GC when marking the reference as not active
+            // assert r.next == r;
+            if (r instanceof sun.misc.Cleaner) {  // Fast path for sun.misc.Cleaners
+                // unlink from the rest in chunk
+                r.discovered = null;
+                ((sun.misc.Cleaner) r).clean();
+            } else if (r instanceof Cleaner) {    // Submit task(s) for j.l.r.Cleaner(s)
+                // link into the local cleaners list
+                r.discovered = cleaners;
+                cleaners = r;
+                if (++cleanersCount >= CLEANER_CHUNK_SIZE) {
+                    // when chunk of finalizers is full, submit a task
+                    new ReferenceHandling.CleanersHandler(cleaners).submit();
+                    cleaners = null;
+                    cleanersCount = 0;
+                }
+            } else {                              // Enqueue all other references
+                // unlink from the rest in chunk
+                r.discovered = null;
+                ReferenceQueue<?> q = r.queue;
+                if (q != ReferenceQueue.NULL && q.markEnqueued(r)) { // markEnqueued is atomic
+                    if (referenceQueue == null || referenceQueue == q) {
+                        // no queue or same queue -> hook onto the references[Head|Tail] chain
+                        if (referencesHead == null) {
+                            // assert referencesTail == null && referenceQueue == null &&
+                            //        referencesCount == 0 && r.next == r;
+                            referenceQueue = q;
+                            referencesHead = referencesTail = r;
+                        } else {
+                            // assert referencesTail != null && referenceQueue == q &&
+                            //        referencesCount > 0;
+                            r.next = referencesHead;
+                            referencesHead = r;
+                        }
+                        if (++referencesCount >= ENQUEUE_CHUNK_SIZE) {
+                            // when a chunk of references is full, add them to queue
+                            referenceQueue.addChunk(referencesHead, referencesTail);
+                            referencesHead = referencesTail = null;
+                            referenceQueue = null;
+                            referencesCount = 0;
+                        }
+                    } else {
+                        // when a different queue is encountered,
+                        // add collected chunk to it's queue and start collecting
+                        // new batch for new queue...
+                        // assert referenceQueue != null && referenceQueue != q &&
+                        //        referencesHead != null && referencesTail != null &&
+                        //        referencesCount > 0 && r.next == r;
+                        referenceQueue.addChunk(referencesHead, referencesTail);
+                        referenceQueue = q;
+                        referencesHead = referencesTail = r;
+                        referencesCount = 1;
+                    }
+                }
+                // else just drop it on the flor
+            }
+        }
+        // any j.l.r.Cleaner(s) left?
+        if (cleaners != null) {
+            new ReferenceHandling.CleanersHandler(cleaners).submit();
+            cleaners = null;
+            cleanersCount = 0;
+        }
+        // any references left to enqueue?
+        if (referenceQueue != null) {
+            // assert referencesHead != null && referencesTail != null && referencesCount > 0;
+            referenceQueue.addChunk(referencesHead, referencesTail);
+            referencesHead = referencesTail = null;
+            referenceQueue = null;
+            referencesCount = 0;
+        }
+    }
 
-        // provide access in SharedSecrets
-        SharedSecrets.setJavaLangRefAccess(new JavaLangRefAccess() {
-            @Override
-            public boolean tryHandlePendingReference() {
-                return tryHandlePending(false);
+    /**
+     * Handles a non-null chunk of j.l.r.Cleaner(s)
+     *
+     * @param cleaners the head of a chunk of Reference(s) implementing j.l.r.Cleaner
+     *                 linked using {@link #discovered} link. Last in chunk is
+     *                 linked to null.
+     */
+    static void handleCleaners(Reference<?> cleaners) {
+        for (Reference<?> c = cleaners, d = c.discovered;
+             c != null;
+             c = d, d = (c == null) ? null : c.discovered) {
+            // unlink it from the rest in chunk
+            c.discovered = null;
+            // Handle it
+            try {
+                ((Cleaner) c).clean();
+            } catch (Throwable t) {
+                // ignore
+            }
             }
-        });
     }
 
     /* -- Referent accessor and setters -- */
 
     /**

@@ -304,9 +452,136 @@
         this(referent, null);
     }
 
     Reference(T referent, ReferenceQueue<? super T> queue) {
         this.referent = referent;
+        if (this instanceof Cleaner && queue != null) {
+            throw new IllegalArgumentException(
+                "Reference implementing Cleaner can't be registered with a reference queue");
+        }
         this.queue = (queue == null) ? ReferenceQueue.NULL : queue;
     }
 
+    // Methods that enable selected Reference subclasses to be elements of a DLList
+
+    /**
+     * Some Cleaner's (such as Finalizer) wish to be registered in a doubly-linked
+     * list so that they are kept alive until discovered by VM, processed by
+     * ReferenceHandling threads and then unlinked. There are several lists to
+     * distribute them randomly into to reduce contention among concurrent threads
+     * trying to link/unlink them.
+     */
+    static final DLList[] uncleaned;
+
+    boolean isDeletedDll() {
+        throw new UnsupportedOperationException();
+    }
+
+    Reference<?> getPrevDll() {
+        throw new UnsupportedOperationException();
+    }
+
+    void lazySetPrevDll(Reference<?> val) {
+        throw new UnsupportedOperationException();
+    }
+
+    boolean casPrevDll(Reference<?> cmp, Reference<?> val) {
+        throw new UnsupportedOperationException();
+    }
+
+    Reference<?> getNextDll() {
+        throw new UnsupportedOperationException();
+    }
+
+    void lazySetNextDll(Reference<?> val) {
+        throw new UnsupportedOperationException();
+    }
+
+    boolean casNextDll(Reference<?> cmp, Reference<?> val) {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Returns the pseudo-randomly initialized or updated secondary seed.
+     * Copied from ThreadLocalRandom due to package access restrictions.
+     */
+    static int nextSecondarySeed() {
+        int r;
+        Thread t = Thread.currentThread();
+        if ((r = UNSAFE.getInt(t, threadLocalRandomSecondarySeedOffset)) != 0) {
+            r ^= r << 13;   // xorshift
+            r ^= r >>> 17;
+            r ^= r << 5;
+        }
+        else if ((r = ThreadLocalRandom.current().nextInt()) == 0)
+            r = 1; // avoid zero
+        UNSAFE.putInt(t, threadLocalRandomSecondarySeedOffset, r);
+        return r;
+    }
+
+    // Unsafe machinery
+
+    @SuppressWarnings("unchecked")
+    T getReferentVolatile() {
+        return (T) UNSAFE.getObjectVolatile(this, referentOffset);
+    }
+
+    boolean casReferent(T cmp, T val) {
+        return UNSAFE.compareAndSwapObject(this, referentOffset, cmp, val);
+    }
+
+    void lazySetQueue(ReferenceQueue<? super T> val) {
+        UNSAFE.putOrderedObject(this, queueOffset, val);
+    }
+
+    boolean casQueue(ReferenceQueue<?> cmp, ReferenceQueue<? super T> val) {
+        return UNSAFE.compareAndSwapObject(this, queueOffset, cmp, val);
+    }
+
+    private static final sun.misc.Unsafe UNSAFE;
+    private static final long referentOffset;
+    private static final long queueOffset;
+    private static final long threadLocalRandomSecondarySeedOffset;
+
+    static {
+        try {
+            UNSAFE = sun.misc.Unsafe.getUnsafe();
+            Class<Reference> rc = Reference.class;
+            referentOffset = UNSAFE.objectFieldOffset(rc.getDeclaredField("referent"));
+            queueOffset = UNSAFE.objectFieldOffset(rc.getDeclaredField("queue"));
+            Class<Thread> tc = Thread.class;
+            threadLocalRandomSecondarySeedOffset = UNSAFE.objectFieldOffset
+                (tc.getDeclaredField("threadLocalRandomSecondarySeed"));
+        } catch (Exception e) {
+            throw new Error(e);
+        }
+
+        // DLList(s) for selected uncleaned Cleaner(s)
+        int cpus = Runtime.getRuntime().availableProcessors();
+        // smallest power of two equal or greater than 2 * # of CPUs
+        int lists = (cpus <= 1) ? 2 : Integer.highestOneBit(cpus - 1) << 2;
+        uncleaned = new DLList[lists];
+        for (int i = 0; i < uncleaned.length; i++) {
+            uncleaned[i] = new DLList();
+        }
+
+        ThreadGroup tg = Thread.currentThread().getThreadGroup();
+        for (ThreadGroup tgn = tg;
+             tgn != null;
+             tg = tgn, tgn = tg.getParent());
+        Thread handler = new ReferenceHandler(tg, "Reference Handler");
+        /* If there were a special system-only priority greater than
+         * MAX_PRIORITY, it would be used here
+         */
+        handler.setPriority(Thread.MAX_PRIORITY);
+        handler.setDaemon(true);
+        handler.start();
+
+        // provide access in SharedSecrets
+        SharedSecrets.setJavaLangRefAccess(new JavaLangRefAccess() {
+            @Override
+            public boolean tryHandlePendingReference() {
+                return tryHandlePending();
+            }
+        });
+    }
 }
< prev index next >