--- old/src/java.base/share/classes/java/lang/ref/Finalizer.java 2015-06-06 15:34:10.680931393 +0200 +++ new/src/java.base/share/classes/java/lang/ref/Finalizer.java 2015-06-06 15:34:10.590932969 +0200 @@ -27,98 +27,101 @@ import java.security.PrivilegedAction; import java.security.AccessController; -import sun.misc.JavaLangAccess; +import java.util.concurrent.ThreadLocalRandom; import sun.misc.ManagedLocalsThread; import sun.misc.SharedSecrets; import sun.misc.VM; -final class Finalizer extends FinalReference { /* Package-private; must be in - same package as the Reference - class */ - - private static ReferenceQueue queue = new ReferenceQueue<>(); - private static Finalizer unfinalized = null; - private static final Object lock = new Object(); - - private Finalizer - next = null, - prev = null; - - private boolean hasBeenFinalized() { - return (next == this); - } - - private void add() { - synchronized (lock) { - if (unfinalized != null) { - this.next = unfinalized; - unfinalized.prev = this; - } - unfinalized = this; +/* Package-private; must be in same package as the Reference class */ +class Finalizer extends FinalReference implements Runnable { + /** + * Finalizers are registered in a doubly-linked list so that they are kept + * alive until discovered by VM, processed by ReferenceHandling pool and then + * unlinked. There are several lists to distribute Finalizers randomly into + * to reduce contention among concurrent threads trying to link/unlink them. + */ + static final FinalizerList[] unfinalized; + static { + 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; + unfinalized = new FinalizerList[lists]; + for (int i = 0; i < lists; i++) { + unfinalized[i] = new FinalizerList(); } } - private void remove() { - synchronized (lock) { - if (unfinalized == this) { - if (this.next != null) { - unfinalized = this.next; - } else { - unfinalized = this.prev; - } - } - if (this.next != null) { - this.next.prev = this.prev; - } - if (this.prev != null) { - this.prev.next = this.next; - } - this.next = this; /* Indicates that this has been finalized */ - this.prev = this; - } + volatile Finalizer prev; + volatile Finalizer next; + private final int listIndex; + + Finalizer(T finalizee, int listIndex) { + super(finalizee, ReferenceQueue.NULL); + this.listIndex = listIndex; } - private Finalizer(Object finalizee) { - super(finalizee, queue); - add(); + /** A constructor used for special Finalizer instances in FinalizerList */ + Finalizer() { + super(null, ReferenceQueue.NULL); + listIndex = -1; // never registered in any list } - /* Invoked by VM */ + /** Invoked by VM for objects overriding finalize() method */ static void register(Object finalizee) { - new Finalizer(finalizee); + int rnd = nextSecondarySeed(); + int index = (rnd >>> 1) & (unfinalized.length - 1); + unfinalized[index].link(new Finalizer<>(finalizee, index), (rnd & 1) == 0); } - private void runFinalizer(JavaLangAccess jla) { - synchronized (this) { - if (hasBeenFinalized()) return; - remove(); + @Override + public void run() { + T finalizee = delete(); + if (finalizee == null) { + return; } + unfinalized[listIndex].unlink(this); try { - Object finalizee = this.get(); - if (finalizee != null && !(finalizee instanceof java.lang.Enum)) { - jla.invokeFinalize(finalizee); + if (!(finalizee instanceof java.lang.Enum)) { + invokeFinalizee(finalizee); /* Clear stack slot containing this variable, to decrease the chances of false retention with a conservative GC */ finalizee = null; } } catch (Throwable x) { } - super.clear(); + } + + /* Invoke the finalize() method on the finalizee (overridden by Finalizator) */ + void invokeFinalizee(T finalizee) throws Throwable { + SharedSecrets.getJavaLangAccess().invokeFinalize(finalizee); + finalizee = null; + } + + @Override + public void clear() { + T finalizee = delete(); + if (finalizee == null) { + return; + } + unfinalized[listIndex].unlink(this); + /* Clear stack slot containing this variable, to decrease + the chances of false retention with a conservative GC */ + finalizee = null; } /* Create a privileged secondary finalizer thread in the system thread - group for the given Runnable, and wait for it to complete. + group for the given Runnable, and wait for it to complete. - This method is used by both runFinalization and runFinalizersOnExit. - The former method invokes all pending finalizers, while the latter - invokes all uninvoked finalizers if on-exit finalization has been - enabled. - - These two methods could have been implemented by offloading their work - to the regular finalizer thread and waiting for that thread to finish. - The advantage of creating a fresh thread, however, is that it insulates - invokers of these methods from a stalled or deadlocked finalizer thread. - */ + This method is used by both runFinalization and runFinalizersOnExit. + The former method invokes all pending finalizers, while the latter + invokes all uninvoked finalizers if on-exit finalization has been + enabled. + + These two methods could have been implemented by offloading their work + to the regular finalizer thread and waiting for that thread to finish. + The advantage of creating a fresh thread, however, is that it insulates + invokers of these methods from a stalled or deadlocked finalizer thread. + */ private static void forkSecondaryFinalizer(final Runnable proc) { AccessController.doPrivileged( new PrivilegedAction<>() { @@ -150,13 +153,8 @@ // in case of recursive call to run() if (running) return; - final JavaLangAccess jla = SharedSecrets.getJavaLangAccess(); running = true; - for (;;) { - Finalizer f = (Finalizer)queue.poll(); - if (f == null) break; - f.runFinalizer(jla); - } + ReferenceHandling.runFinalization(); } }); } @@ -173,61 +171,79 @@ // in case of recursive call to run() if (running) return; - final JavaLangAccess jla = SharedSecrets.getJavaLangAccess(); running = true; - for (;;) { - Finalizer f; - synchronized (lock) { - f = unfinalized; - if (f == null) break; - unfinalized = f.next; - } - f.runFinalizer(jla); + for (FinalizerList uflist : unfinalized) + for (Finalizer f = uflist.first(); f != null; f = uflist.succ(f)) { + f.run(); }}}); } - private static class FinalizerThread extends ManagedLocalsThread { - private volatile boolean running; - FinalizerThread(ThreadGroup g) { - super(g, "Finalizer"); - } - public void run() { - // in case of recursive call to run() - if (running) - return; - - // Finalizer thread starts before System.initializeSystemClass - // is called. Wait until JavaLangAccess is available - while (!VM.isBooted()) { - // delay until VM completes initialization - try { - VM.awaitBooted(); - } catch (InterruptedException x) { - // ignore and continue - } - } - final JavaLangAccess jla = SharedSecrets.getJavaLangAccess(); - running = true; - for (;;) { - try { - Finalizer f = (Finalizer)queue.remove(); - f.runFinalizer(jla); - } catch (InterruptedException x) { - // ignore and continue - } - } + // Unsafe mechanics + + /** + * 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; } - static { - ThreadGroup tg = Thread.currentThread().getThreadGroup(); - for (ThreadGroup tgn = tg; - tgn != null; - tg = tgn, tgn = tg.getParent()); - Thread finalizer = new FinalizerThread(tg); - finalizer.setPriority(Thread.MAX_PRIORITY - 2); - finalizer.setDaemon(true); - finalizer.start(); + boolean isAlive() { + return getReferentVolatile() != null; + } + + boolean isDeleted() { + return getReferentVolatile() == null; + } + + private T delete() { + T referent = getReferentVolatile(); + return (referent != null) && casReferent(referent, null) + ? referent : null; + } + + void lazySetNext(Finalizer val) { + UNSAFE.putOrderedObject(this, nextOffset, val); + } + + boolean casNext(Finalizer cmp, Finalizer val) { + return UNSAFE.compareAndSwapObject(this, nextOffset, cmp, val); } + void lazySetPrev(Finalizer val) { + UNSAFE.putOrderedObject(this, prevOffset, val); + } + + boolean casPrev(Finalizer cmp, Finalizer val) { + return UNSAFE.compareAndSwapObject(this, prevOffset, cmp, val); + } + + private static final sun.misc.Unsafe UNSAFE; + private static final long prevOffset; + private static final long nextOffset; + private static final long threadLocalRandomSecondarySeedOffset; + + static { + try { + UNSAFE = sun.misc.Unsafe.getUnsafe(); + Class fc = Finalizer.class; + prevOffset = UNSAFE.objectFieldOffset(fc.getDeclaredField("prev")); + nextOffset = UNSAFE.objectFieldOffset(fc.getDeclaredField("next")); + Class tc = Thread.class; + threadLocalRandomSecondarySeedOffset = UNSAFE.objectFieldOffset + (tc.getDeclaredField("threadLocalRandomSecondarySeed")); + } catch (Exception e) { + throw new Error(e); + } + } } --- old/src/java.base/share/classes/java/lang/ref/Reference.java 2015-06-06 15:34:11.016925510 +0200 +++ new/src/java.base/share/classes/java/lang/ref/Reference.java 2015-06-06 15:34:10.916927261 +0200 @@ -99,14 +99,13 @@ * 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 discovered; /* used by VM */ + transient private Reference discovered; /* used by VM */ /* Object used to synchronize with the garbage collector. The collector @@ -115,7 +114,7 @@ * 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 @@ -125,8 +124,20 @@ */ private static Reference pending = null; - /* High-priority thread to enqueue pending References + /* 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; + + /* Max. number of Finalizer(s) to execute in one ForkJoinTask + */ + private static final int FINALIZE_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) { @@ -150,95 +161,208 @@ } 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; + } + } + // start reference handling ForkJoinPool + ReferenceHandling.start(); + // 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.

- * 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.

+ * 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 r; - Cleaner c; - try { - 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(); - } - // retry if waited - return waitForNotify; + * @return {@code true} if there is be more {@link Reference}s pending. + */ + static boolean tryHandlePending() { + Reference r; + synchronized (lock) { + r = unhookPendingChunk(UNHOOK_CHUNK_SIZE, null); + } + if (r == null) return false; + handlePendingChunk(r); + synchronized (lock) { + return pending != null; + } + } + + /** + * 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 #next} links; the last in chunk is linked to itself. + * + * @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) { + // pending state invariant established by VM: + // assert r.next == r; + // move a chunk of pending/discovered references to a + // temporary local r/next chain + Reference rd = r.discovered; + for (int i = 0; rd != null; rd = r.discovered) { + r.discovered = null; + if (++i >= chunkSize) { + break; } + rd.next = r; + r = rd; } - } 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 q = r.queue; - if (q != ReferenceQueue.NULL) q.enqueue(r); - return true; + pending = (Reference) rd; + if (morePending != null) morePending[0] = (rd != 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(); - - // provide access in SharedSecrets - SharedSecrets.setJavaLangRefAccess(new JavaLangRefAccess() { - @Override - public boolean tryHandlePendingReference() { - return tryHandlePending(false); + /** + * Handles a non-null chunk of pending references + * (obtained using {@link #unhookPendingChunk}) and handles + * them as following: + *
    + *
  • Cleaner(s) are executed immediately
  • + *
  • Finalizer(s) are submitted as ForkJoinTask(s)
  • + *
  • all other Reference(s) are enqueued in their respected queues
  • + *
+ * @param chunk the head of a chunk of pending references + */ + static void handlePendingChunk(Reference chunk) { + // batch finaliz(ato|e)rs and Finalizators + Reference finalizrs = null; + int finalizrsCount = 0; + // batch consecutive references with same queue into chunks + Reference referencesHead = null, referencesTail = null; + int referencesCount = 0; + ReferenceQueue referenceQueue = null; + // dispatch references to appropriate targets + for (Reference r = chunk, rn = r.next; ; r = rn, rn = r.next) { + if (r instanceof Cleaner) { // Fast path for cleaners + // take 'r' off the chain + r.next = r; + ((Cleaner) r).clean(); + } else if (r instanceof Finalizer || + r instanceof Finalizator) { // Submit task(s) for finaliz(ato|e)rs + // hook onto the finalizers chain + r.next = (finalizrs == null) ? r : finalizrs; + finalizrs = r; + if (++finalizrsCount >= FINALIZE_CHUNK_SIZE) { + // when chunk of finaliz(ato|e)rs is full, submit a task + new ReferenceHandling.FinalizrHandler(finalizrs).submit(); + finalizrs = null; + finalizrsCount = 0; + } + } else { // Enqueue all other references + // take 'r' off the chain + r.next = r; + 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 + // into 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; + } + } } - }); + if (rn == r) { // last in chain + break; + } + } + // any finalizers left? + if (finalizrs != null) { + new ReferenceHandling.FinalizrHandler(finalizrs).submit(); + finalizrs = null; + finalizrsCount = 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; + } } /* -- Referent accessor and setters -- */ @@ -309,4 +433,57 @@ this.queue = (queue == null) ? ReferenceQueue.NULL : queue; } + // 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 val) { + UNSAFE.putOrderedObject(this, queueOffset, val); + } + + boolean casQueue(ReferenceQueue cmp, ReferenceQueue 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; + + static { + try { + UNSAFE = sun.misc.Unsafe.getUnsafe(); + Class rc = Reference.class; + referentOffset = UNSAFE.objectFieldOffset(rc.getDeclaredField("referent")); + queueOffset = UNSAFE.objectFieldOffset(rc.getDeclaredField("queue")); + } catch (Exception e) { + throw new Error(e); + } + + 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(); + } + }); + } } --- old/src/java.base/share/classes/java/lang/ref/ReferenceQueue.java 2015-06-06 15:34:11.377919188 +0200 +++ new/src/java.base/share/classes/java/lang/ref/ReferenceQueue.java 2015-06-06 15:34:11.276920957 +0200 @@ -46,51 +46,70 @@ } } - static ReferenceQueue NULL = new Null<>(); - static ReferenceQueue ENQUEUED = new Null<>(); + static final ReferenceQueue NULL = new Null<>(); + static final ReferenceQueue ENQUEUED = new Null<>(); - static private class Lock { }; - private Lock lock = new Lock(); - private volatile Reference head = null; - private long queueLength = 0; + static private class Lock { } + private final Lock lock = new Lock(); + private volatile int waiters; + + @SuppressWarnings("unused") + private volatile Reference head; // we assign using Unsafe CAS boolean enqueue(Reference r) { /* Called only by Reference class */ - synchronized (lock) { - // Check that since getting the lock this reference hasn't already been + if (markEnqueued(r)) { + addChunk(r, r); + return true; + } else { + return false; + } + } + + boolean markEnqueued(Reference r) { + ReferenceQueue queue; + do { + // Check that this reference hasn't already been // enqueued (and even then removed) - ReferenceQueue queue = r.queue; + queue = r.queue; if ((queue == NULL) || (queue == ENQUEUED)) { return false; } - assert queue == this; - r.queue = ENQUEUED; - r.next = (head == null) ? r : head; - head = r; - queueLength++; - if (r instanceof FinalReference) { - sun.misc.VM.addFinalRefCount(1); + } while (!r.casQueue(queue, ENQUEUED)); + assert queue == this; + return true; + } + + @SuppressWarnings("unchecked") + void addChunk(Reference chunkHead, Reference chunkTail) { + Reference h; + do { + h = head; + chunkTail.next = (h == null) ? chunkTail : h; + } while (!casHead(h, (Reference) chunkHead)); + // notify waiters + if (waiters > 0) { + synchronized (lock) { + if (waiters > 0) { + lock.notifyAll(); + } } - lock.notifyAll(); - return true; } } - @SuppressWarnings("unchecked") - private Reference reallyPoll() { /* Must hold lock */ - Reference r = head; - if (r != null) { - head = (r.next == r) ? - null : - r.next; // Unchecked due to the next field having a raw type in Reference - r.queue = NULL; - r.next = r; - queueLength--; - if (r instanceof FinalReference) { - sun.misc.VM.addFinalRefCount(-1); + private Reference reallyPoll() { + Reference r; + while ((r = head) != null) { + @SuppressWarnings("unchecked") // due to cast to raw type + Reference nh = (r.next == r) + ? null : (Reference) r.next; + if (casHead(r, nh)) { + r.lazySetQueue(NULL); + UNSAFE.storeFence(); + r.next = r; + break; } - return r; } - return null; + return r; } /** @@ -102,11 +121,7 @@ * otherwise null */ public Reference poll() { - if (head == null) - return null; - synchronized (lock) { - return reallyPoll(); - } + return reallyPoll(); } /** @@ -135,20 +150,32 @@ if (timeout < 0) { throw new IllegalArgumentException("Negative timeout value"); } + Reference r = reallyPoll(); + if (r != null) return r; + return reallyRemove(timeout); + } + + private Reference reallyRemove(long timeout) throws InterruptedException { + long deadline = (timeout == 0) + ? 0 : System.nanoTime() + timeout * 1000_000L; + synchronized (lock) { - Reference r = reallyPoll(); - if (r != null) return r; - long start = (timeout == 0) ? 0 : System.nanoTime(); - for (;;) { - lock.wait(timeout); - r = reallyPoll(); - if (r != null) return r; - if (timeout != 0) { - long end = System.nanoTime(); - timeout -= (end - start) / 1000_000; - if (timeout <= 0) return null; - start = end; + int w = waiters; + waiters = w + 1; + try { + for (; ; ) { + Reference r = reallyPoll(); + if (r != null) return r; + if (timeout == 0) { + lock.wait(0); + } else { + long timeoutNanos = deadline - System.nanoTime(); + if (timeoutNanos <= 0) return null; + lock.wait(timeoutNanos / 1000_000L, (int) (timeoutNanos % 1000_000L)); + } } + } finally { + waiters = w; } } } @@ -164,4 +191,22 @@ return remove(0); } + // Unsafe machinery + + private boolean casHead(Reference cmp, Reference val) { + return UNSAFE.compareAndSwapObject(this, headOffset, cmp, val); + } + + private static final sun.misc.Unsafe UNSAFE; + private static final long headOffset; + + static { + try { + UNSAFE = sun.misc.Unsafe.getUnsafe(); + Class rqc = ReferenceQueue.class; + headOffset = UNSAFE.objectFieldOffset(rqc.getDeclaredField("head")); + } catch (Exception e) { + throw new Error(e); + } + } } --- /dev/null 2015-06-06 09:45:45.596162458 +0200 +++ new/src/java.base/share/classes/java/lang/ref/Finalizator.java 2015-06-06 15:34:11.577915687 +0200 @@ -0,0 +1,163 @@ +package java.lang.ref; + +import java.util.Objects; +import java.util.function.Consumer; + +/** + *

Finalizator provides an alternative form of finalization which can be more + * efficiently combined with (almost) regular manual cleanup where it serves as + * a last-resort cleanup mechanism when manual cleanup is not performed. + *

+ * Instead of overriding the Object's {@link #finalize()} method, a class + * arranges in it's constructor(s) to {@link #create(Object, Consumer) create} a + * {@code Finalizator} object which is used by GC to track the reachability of + * given {@code finalizee} and invoke given {@code thunk} with it when such + * {@code finalizee} becomes unreachable. + *

+ * {@code Finalizator} can be {@link #run() invoked} manually by user code that + * decides to perform cleanup even before GC invokes it. The 1st invocation of + * {@link #run()} method performs the cleanup by invoking the {@code thunk}, + * subsequent invocations are ignored. + *

+ * After the cleanup is performed, the + * {@code Finalizator} is {@link #clear() cleared} which breaks the links between: + *

    + *
  • finalizator and finalizee,
  • + *
  • finalizator and thunk,
  • + *
  • platform and finalizator
  • + *
+ * In the absence of other links, finalizator, finalizee and thunk become + * unreachable and eligible for GC. + *

+ * Here's an example of a classic finalizable class: + *

{@code
+ *      public class Classic {
+ *          @Override
+ *          protected void finalize() {
+ *              // clean-up actions invoked at most once...
+ *          }
+ *      }
+ * }
+ *

+ * And this is an alternative using {@code Finalizator}, combining finalization + * with manual cleanup: + *

{@code
+ *      public class Alternative {
+ *          private final Finalizator finalizator =
+ *              Finalizator.create(this, Alternative::cleanup);
+ *
+ *          void cleanup() {
+ *              // clean-up actions invoked at most once...
+ *          }
+ *
+ *          // manually triggered cleanup
+ *          public void close() {
+ *              finalizator.run();
+ *          }
+ *      }
+ * }
+ * + * @param the type of finalizee tracked by Finalizator + * @since 1.9 + */ +public final class Finalizator extends Finalizer implements Runnable { + + private Consumer thunk; + + /** + * Creates and returns an instance of Finalizator used by GC to track given + * {@code finalizee} and invoke given {@code thunk} with it when the + * finalizee becomes unreachable. + * + * @param finalizee the instance to track it's reachability + * @param thunk a {@link Consumer} which is invoked and passed the + * {@code finalizee} when it becomes unreachable. + * @throws NullPointerException if either {@code finalizee} of {@code thunk} + * are null + */ + public static Finalizator create(T finalizee, Consumer thunk) { + Objects.requireNonNull(finalizee); + Objects.requireNonNull(thunk); + int rnd = nextSecondarySeed(); + int index = (rnd >>> 1) & (unfinalized.length - 1); + Finalizator finalizator = new Finalizator<>(finalizee, index, thunk); + unfinalized[index].link(finalizator, (rnd & 1) == 0); + return finalizator; + } + + private Finalizator(T finalizee, int listIndex, Consumer thunk) { + super(finalizee, listIndex); + this.thunk = thunk; + } + + /** + * Invoked by GC when the tracked finalizee becomes unreachable or + * by user code at any time. + * It invokes the finalizator's thunk with the finalizee if this + * Finalizator has not been {@link #clear() cleared} before that. If invoked + * multiple times, only the 1st invocation results in the invocation + * of the thunk. The finalizator releases the reference to the thunk and + * finalizee upon 1st invocation of this method regardless of whether + * it was performed by GC or by user code. + */ + @Override + public void run() { + super.run(); + } + + /** + * Invoke the {@link #thunk} passing the {@code finalizee} to it. + */ + @Override + void invokeFinalizee(T finalizee) throws Throwable { + Consumer thunk = this.thunk; + this.thunk = null; + thunk.accept(finalizee); + finalizee = null; + } + + /** + * Returns {@code null} to prevent unwanted retention of the tracked + * {@code finalizee}. + * + * @return {@code null} + */ + @Override + public T get() { + return null; + } + + /** + * Finalizator is not registered with a reference queue when created, + * so this method always returns {@code false}. + * + * @return {@code false} + */ + @Override + public boolean isEnqueued() { + return false; + } + + /** + * Finalizator is not registered with a reference queue when created, + * so this method does nothing and always returns {@code false}. + * + * @return {@code false} + */ + @Override + public boolean enqueue() { + return false; + } + + /** + * Clears this Finalizator and releases it's tracked {@code finalizee} and + * it's {@code thunk} if they have not been released yet. + * Invoking this method does not invoke the Finalizator's {@code thunk}. + * It prevents from {@link #run() running} the {@code thunk} in the future + * if it has not been run yet. + */ + @Override + public void clear() { + super.clear(); + } +} --- old/src/java.base/share/classes/java/util/concurrent/ConcurrentLinkedDeque.java 2015-06-06 15:34:11.934909436 +0200 +++ /dev/null 2015-06-06 09:45:45.596162458 +0200 @@ -1,1586 +0,0 @@ -/* - * 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. Oracle designates this - * particular file as subject to the "Classpath" exception as provided - * by Oracle in the LICENSE file that accompanied this code. - * - * This code is distributed in the hope that it will be useful, but WITHOUT - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or - * FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License - * version 2 for more details (a copy is included in the LICENSE file that - * accompanied this code). - * - * You should have received a copy of the GNU General Public License version - * 2 along with this work; if not, write to the Free Software Foundation, - * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA. - * - * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA - * or visit www.oracle.com if you need additional information or have any - * questions. - */ - -/* - * This file is available under and governed by the GNU General Public - * License version 2 only, as published by the Free Software Foundation. - * However, the following notice accompanied the original version of this - * file: - * - * Written by Doug Lea and Martin Buchholz with assistance from members of - * JCP JSR-166 Expert Group and released to the public domain, as explained - * at http://creativecommons.org/publicdomain/zero/1.0/ - */ - -package java.util.concurrent; - -import java.util.AbstractCollection; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Deque; -import java.util.Iterator; -import java.util.NoSuchElementException; -import java.util.Queue; -import java.util.Spliterator; -import java.util.Spliterators; -import java.util.function.Consumer; - -/** - * An unbounded concurrent {@linkplain Deque deque} based on linked nodes. - * Concurrent insertion, removal, and access operations execute safely - * across multiple threads. - * A {@code ConcurrentLinkedDeque} is an appropriate choice when - * many threads will share access to a common collection. - * Like most other concurrent collection implementations, this class - * does not permit the use of {@code null} elements. - * - *

Iterators and spliterators are - * weakly consistent. - * - *

Beware that, unlike in most collections, the {@code size} method - * is NOT a constant-time operation. Because of the - * asynchronous nature of these deques, determining the current number - * of elements requires a traversal of the elements, and so may report - * inaccurate results if this collection is modified during traversal. - * Additionally, the bulk operations {@code addAll}, - * {@code removeAll}, {@code retainAll}, {@code containsAll}, - * {@code equals}, and {@code toArray} are not guaranteed - * to be performed atomically. For example, an iterator operating - * concurrently with an {@code addAll} operation might view only some - * of the added elements. - * - *

This class and its iterator implement all of the optional - * methods of the {@link Deque} and {@link Iterator} interfaces. - * - *

Memory consistency effects: As with other concurrent collections, - * actions in a thread prior to placing an object into a - * {@code ConcurrentLinkedDeque} - * happen-before - * actions subsequent to the access or removal of that element from - * the {@code ConcurrentLinkedDeque} in another thread. - * - *

This class is a member of the - * - * Java Collections Framework. - * - * @since 1.7 - * @author Doug Lea - * @author Martin Buchholz - * @param the type of elements held in this collection - */ -public class ConcurrentLinkedDeque - extends AbstractCollection - implements Deque, java.io.Serializable { - - /* - * This is an implementation of a concurrent lock-free deque - * supporting interior removes but not interior insertions, as - * required to support the entire Deque interface. - * - * We extend the techniques developed for ConcurrentLinkedQueue and - * LinkedTransferQueue (see the internal docs for those classes). - * Understanding the ConcurrentLinkedQueue implementation is a - * prerequisite for understanding the implementation of this class. - * - * The data structure is a symmetrical doubly-linked "GC-robust" - * linked list of nodes. We minimize the number of volatile writes - * using two techniques: advancing multiple hops with a single CAS - * and mixing volatile and non-volatile writes of the same memory - * locations. - * - * A node contains the expected E ("item") and links to predecessor - * ("prev") and successor ("next") nodes: - * - * class Node { volatile Node prev, next; volatile E item; } - * - * A node p is considered "live" if it contains a non-null item - * (p.item != null). When an item is CASed to null, the item is - * atomically logically deleted from the collection. - * - * At any time, there is precisely one "first" node with a null - * prev reference that terminates any chain of prev references - * starting at a live node. Similarly there is precisely one - * "last" node terminating any chain of next references starting at - * a live node. The "first" and "last" nodes may or may not be live. - * The "first" and "last" nodes are always mutually reachable. - * - * A new element is added atomically by CASing the null prev or - * next reference in the first or last node to a fresh node - * containing the element. The element's node atomically becomes - * "live" at that point. - * - * A node is considered "active" if it is a live node, or the - * first or last node. Active nodes cannot be unlinked. - * - * A "self-link" is a next or prev reference that is the same node: - * p.prev == p or p.next == p - * Self-links are used in the node unlinking process. Active nodes - * never have self-links. - * - * A node p is active if and only if: - * - * p.item != null || - * (p.prev == null && p.next != p) || - * (p.next == null && p.prev != p) - * - * The deque object has two node references, "head" and "tail". - * The head and tail are only approximations to the first and last - * nodes of the deque. The first node can always be found by - * following prev pointers from head; likewise for tail. However, - * it is permissible for head and tail to be referring to deleted - * nodes that have been unlinked and so may not be reachable from - * any live node. - * - * There are 3 stages of node deletion; - * "logical deletion", "unlinking", and "gc-unlinking". - * - * 1. "logical deletion" by CASing item to null atomically removes - * the element from the collection, and makes the containing node - * eligible for unlinking. - * - * 2. "unlinking" makes a deleted node unreachable from active - * nodes, and thus eventually reclaimable by GC. Unlinked nodes - * may remain reachable indefinitely from an iterator. - * - * Physical node unlinking is merely an optimization (albeit a - * critical one), and so can be performed at our convenience. At - * any time, the set of live nodes maintained by prev and next - * links are identical, that is, the live nodes found via next - * links from the first node is equal to the elements found via - * prev links from the last node. However, this is not true for - * nodes that have already been logically deleted - such nodes may - * be reachable in one direction only. - * - * 3. "gc-unlinking" takes unlinking further by making active - * nodes unreachable from deleted nodes, making it easier for the - * GC to reclaim future deleted nodes. This step makes the data - * structure "gc-robust", as first described in detail by Boehm - * (http://portal.acm.org/citation.cfm?doid=503272.503282). - * - * GC-unlinked nodes may remain reachable indefinitely from an - * iterator, but unlike unlinked nodes, are never reachable from - * head or tail. - * - * Making the data structure GC-robust will eliminate the risk of - * unbounded memory retention with conservative GCs and is likely - * to improve performance with generational GCs. - * - * When a node is dequeued at either end, e.g. via poll(), we would - * like to break any references from the node to active nodes. We - * develop further the use of self-links that was very effective in - * other concurrent collection classes. The idea is to replace - * prev and next pointers with special values that are interpreted - * to mean off-the-list-at-one-end. These are approximations, but - * good enough to preserve the properties we want in our - * traversals, e.g. we guarantee that a traversal will never visit - * the same element twice, but we don't guarantee whether a - * traversal that runs out of elements will be able to see more - * elements later after enqueues at that end. Doing gc-unlinking - * safely is particularly tricky, since any node can be in use - * indefinitely (for example by an iterator). We must ensure that - * the nodes pointed at by head/tail never get gc-unlinked, since - * head/tail are needed to get "back on track" by other nodes that - * are gc-unlinked. gc-unlinking accounts for much of the - * implementation complexity. - * - * Since neither unlinking nor gc-unlinking are necessary for - * correctness, there are many implementation choices regarding - * frequency (eagerness) of these operations. Since volatile - * reads are likely to be much cheaper than CASes, saving CASes by - * unlinking multiple adjacent nodes at a time may be a win. - * gc-unlinking can be performed rarely and still be effective, - * since it is most important that long chains of deleted nodes - * are occasionally broken. - * - * The actual representation we use is that p.next == p means to - * goto the first node (which in turn is reached by following prev - * pointers from head), and p.next == null && p.prev == p means - * that the iteration is at an end and that p is a (static final) - * dummy node, NEXT_TERMINATOR, and not the last active node. - * Finishing the iteration when encountering such a TERMINATOR is - * good enough for read-only traversals, so such traversals can use - * p.next == null as the termination condition. When we need to - * find the last (active) node, for enqueueing a new node, we need - * to check whether we have reached a TERMINATOR node; if so, - * restart traversal from tail. - * - * The implementation is completely directionally symmetrical, - * except that most public methods that iterate through the list - * follow next pointers ("forward" direction). - * - * We believe (without full proof) that all single-element deque - * operations (e.g., addFirst, peekLast, pollLast) are linearizable - * (see Herlihy and Shavit's book). However, some combinations of - * operations are known not to be linearizable. In particular, - * when an addFirst(A) is racing with pollFirst() removing B, it is - * possible for an observer iterating over the elements to observe - * A B C and subsequently observe A C, even though no interior - * removes are ever performed. Nevertheless, iterators behave - * reasonably, providing the "weakly consistent" guarantees. - * - * Empirically, microbenchmarks suggest that this class adds about - * 40% overhead relative to ConcurrentLinkedQueue, which feels as - * good as we can hope for. - */ - - private static final long serialVersionUID = 876323262645176354L; - - /** - * A node from which the first node on list (that is, the unique node p - * with p.prev == null && p.next != p) can be reached in O(1) time. - * Invariants: - * - the first node is always O(1) reachable from head via prev links - * - all live nodes are reachable from the first node via succ() - * - head != null - * - (tmp = head).next != tmp || tmp != head - * - head is never gc-unlinked (but may be unlinked) - * Non-invariants: - * - head.item may or may not be null - * - head may not be reachable from the first or last node, or from tail - */ - private transient volatile Node head; - - /** - * A node from which the last node on list (that is, the unique node p - * with p.next == null && p.prev != p) can be reached in O(1) time. - * Invariants: - * - the last node is always O(1) reachable from tail via next links - * - all live nodes are reachable from the last node via pred() - * - tail != null - * - tail is never gc-unlinked (but may be unlinked) - * Non-invariants: - * - tail.item may or may not be null - * - tail may not be reachable from the first or last node, or from head - */ - private transient volatile Node tail; - - private static final Node PREV_TERMINATOR, NEXT_TERMINATOR; - - @SuppressWarnings("unchecked") - Node prevTerminator() { - return (Node) PREV_TERMINATOR; - } - - @SuppressWarnings("unchecked") - Node nextTerminator() { - return (Node) NEXT_TERMINATOR; - } - - static final class Node { - volatile Node prev; - volatile E item; - volatile Node next; - - Node() { // default constructor for NEXT_TERMINATOR, PREV_TERMINATOR - } - - /** - * Constructs a new node. Uses relaxed write because item can - * only be seen after publication via casNext or casPrev. - */ - Node(E item) { - UNSAFE.putObject(this, itemOffset, item); - } - - boolean casItem(E cmp, E val) { - return UNSAFE.compareAndSwapObject(this, itemOffset, cmp, val); - } - - void lazySetNext(Node val) { - UNSAFE.putOrderedObject(this, nextOffset, val); - } - - boolean casNext(Node cmp, Node val) { - return UNSAFE.compareAndSwapObject(this, nextOffset, cmp, val); - } - - void lazySetPrev(Node val) { - UNSAFE.putOrderedObject(this, prevOffset, val); - } - - boolean casPrev(Node cmp, Node val) { - return UNSAFE.compareAndSwapObject(this, prevOffset, cmp, val); - } - - // Unsafe mechanics - - private static final sun.misc.Unsafe UNSAFE; - private static final long prevOffset; - private static final long itemOffset; - private static final long nextOffset; - - static { - try { - UNSAFE = sun.misc.Unsafe.getUnsafe(); - Class k = Node.class; - prevOffset = UNSAFE.objectFieldOffset - (k.getDeclaredField("prev")); - itemOffset = UNSAFE.objectFieldOffset - (k.getDeclaredField("item")); - nextOffset = UNSAFE.objectFieldOffset - (k.getDeclaredField("next")); - } catch (Exception e) { - throw new Error(e); - } - } - } - - /** - * Links e as first element. - */ - private void linkFirst(E e) { - checkNotNull(e); - final Node newNode = new Node(e); - - restartFromHead: - for (;;) - for (Node h = head, p = h, q;;) { - if ((q = p.prev) != null && - (q = (p = q).prev) != null) - // Check for head updates every other hop. - // If p == q, we are sure to follow head instead. - p = (h != (h = head)) ? h : q; - else if (p.next == p) // PREV_TERMINATOR - continue restartFromHead; - else { - // p is first node - newNode.lazySetNext(p); // CAS piggyback - if (p.casPrev(null, newNode)) { - // Successful CAS is the linearization point - // for e to become an element of this deque, - // and for newNode to become "live". - if (p != h) // hop two nodes at a time - casHead(h, newNode); // Failure is OK. - return; - } - // Lost CAS race to another thread; re-read prev - } - } - } - - /** - * Links e as last element. - */ - private void linkLast(E e) { - checkNotNull(e); - final Node newNode = new Node(e); - - restartFromTail: - for (;;) - for (Node t = tail, p = t, q;;) { - if ((q = p.next) != null && - (q = (p = q).next) != null) - // Check for tail updates every other hop. - // If p == q, we are sure to follow tail instead. - p = (t != (t = tail)) ? t : q; - else if (p.prev == p) // NEXT_TERMINATOR - continue restartFromTail; - else { - // p is last node - newNode.lazySetPrev(p); // CAS piggyback - if (p.casNext(null, newNode)) { - // Successful CAS is the linearization point - // for e to become an element of this deque, - // and for newNode to become "live". - if (p != t) // hop two nodes at a time - casTail(t, newNode); // Failure is OK. - return; - } - // Lost CAS race to another thread; re-read next - } - } - } - - private static final int HOPS = 2; - - /** - * Unlinks non-null node x. - */ - void unlink(Node x) { - // assert x != null; - // assert x.item == null; - // assert x != PREV_TERMINATOR; - // assert x != NEXT_TERMINATOR; - - final Node prev = x.prev; - final Node next = x.next; - if (prev == null) { - unlinkFirst(x, next); - } else if (next == null) { - unlinkLast(x, prev); - } else { - // Unlink interior node. - // - // This is the common case, since a series of polls at the - // same end will be "interior" removes, except perhaps for - // the first one, since end nodes cannot be unlinked. - // - // At any time, all active nodes are mutually reachable by - // following a sequence of either next or prev pointers. - // - // Our strategy is to find the unique active predecessor - // and successor of x. Try to fix up their links so that - // they point to each other, leaving x unreachable from - // active nodes. If successful, and if x has no live - // predecessor/successor, we additionally try to gc-unlink, - // leaving active nodes unreachable from x, by rechecking - // that the status of predecessor and successor are - // unchanged and ensuring that x is not reachable from - // tail/head, before setting x's prev/next links to their - // logical approximate replacements, self/TERMINATOR. - Node activePred, activeSucc; - boolean isFirst, isLast; - int hops = 1; - - // Find active predecessor - for (Node p = prev; ; ++hops) { - if (p.item != null) { - activePred = p; - isFirst = false; - break; - } - Node q = p.prev; - if (q == null) { - if (p.next == p) - return; - activePred = p; - isFirst = true; - break; - } - else if (p == q) - return; - else - p = q; - } - - // Find active successor - for (Node p = next; ; ++hops) { - if (p.item != null) { - activeSucc = p; - isLast = false; - break; - } - Node q = p.next; - if (q == null) { - if (p.prev == p) - return; - activeSucc = p; - isLast = true; - break; - } - else if (p == q) - return; - else - p = q; - } - - // TODO: better HOP heuristics - if (hops < HOPS - // always squeeze out interior deleted nodes - && (isFirst | isLast)) - return; - - // Squeeze out deleted nodes between activePred and - // activeSucc, including x. - skipDeletedSuccessors(activePred); - skipDeletedPredecessors(activeSucc); - - // Try to gc-unlink, if possible - if ((isFirst | isLast) && - - // Recheck expected state of predecessor and successor - (activePred.next == activeSucc) && - (activeSucc.prev == activePred) && - (isFirst ? activePred.prev == null : activePred.item != null) && - (isLast ? activeSucc.next == null : activeSucc.item != null)) { - - updateHead(); // Ensure x is not reachable from head - updateTail(); // Ensure x is not reachable from tail - - // Finally, actually gc-unlink - x.lazySetPrev(isFirst ? prevTerminator() : x); - x.lazySetNext(isLast ? nextTerminator() : x); - } - } - } - - /** - * Unlinks non-null first node. - */ - private void unlinkFirst(Node first, Node next) { - // assert first != null; - // assert next != null; - // assert first.item == null; - for (Node o = null, p = next, q;;) { - if (p.item != null || (q = p.next) == null) { - if (o != null && p.prev != p && first.casNext(next, p)) { - skipDeletedPredecessors(p); - if (first.prev == null && - (p.next == null || p.item != null) && - p.prev == first) { - - updateHead(); // Ensure o is not reachable from head - updateTail(); // Ensure o is not reachable from tail - - // Finally, actually gc-unlink - o.lazySetNext(o); - o.lazySetPrev(prevTerminator()); - } - } - return; - } - else if (p == q) - return; - else { - o = p; - p = q; - } - } - } - - /** - * Unlinks non-null last node. - */ - private void unlinkLast(Node last, Node prev) { - // assert last != null; - // assert prev != null; - // assert last.item == null; - for (Node o = null, p = prev, q;;) { - if (p.item != null || (q = p.prev) == null) { - if (o != null && p.next != p && last.casPrev(prev, p)) { - skipDeletedSuccessors(p); - if (last.next == null && - (p.prev == null || p.item != null) && - p.next == last) { - - updateHead(); // Ensure o is not reachable from head - updateTail(); // Ensure o is not reachable from tail - - // Finally, actually gc-unlink - o.lazySetPrev(o); - o.lazySetNext(nextTerminator()); - } - } - return; - } - else if (p == q) - return; - else { - o = p; - p = q; - } - } - } - - /** - * Guarantees that any node which was unlinked before a call to - * this method will be unreachable from head after it returns. - * Does not guarantee to eliminate slack, only that head will - * point to a node that was active while this method was running. - */ - private final void updateHead() { - // Either head already points to an active node, or we keep - // trying to cas it to the first node until it does. - Node h, p, q; - restartFromHead: - while ((h = head).item == null && (p = h.prev) != null) { - for (;;) { - if ((q = p.prev) == null || - (q = (p = q).prev) == null) { - // It is possible that p is PREV_TERMINATOR, - // but if so, the CAS is guaranteed to fail. - if (casHead(h, p)) - return; - else - continue restartFromHead; - } - else if (h != head) - continue restartFromHead; - else - p = q; - } - } - } - - /** - * Guarantees that any node which was unlinked before a call to - * this method will be unreachable from tail after it returns. - * Does not guarantee to eliminate slack, only that tail will - * point to a node that was active while this method was running. - */ - private final void updateTail() { - // Either tail already points to an active node, or we keep - // trying to cas it to the last node until it does. - Node t, p, q; - restartFromTail: - while ((t = tail).item == null && (p = t.next) != null) { - for (;;) { - if ((q = p.next) == null || - (q = (p = q).next) == null) { - // It is possible that p is NEXT_TERMINATOR, - // but if so, the CAS is guaranteed to fail. - if (casTail(t, p)) - return; - else - continue restartFromTail; - } - else if (t != tail) - continue restartFromTail; - else - p = q; - } - } - } - - private void skipDeletedPredecessors(Node x) { - whileActive: - do { - Node prev = x.prev; - // assert prev != null; - // assert x != NEXT_TERMINATOR; - // assert x != PREV_TERMINATOR; - Node p = prev; - findActive: - for (;;) { - if (p.item != null) - break findActive; - Node q = p.prev; - if (q == null) { - if (p.next == p) - continue whileActive; - break findActive; - } - else if (p == q) - continue whileActive; - else - p = q; - } - - // found active CAS target - if (prev == p || x.casPrev(prev, p)) - return; - - } while (x.item != null || x.next == null); - } - - private void skipDeletedSuccessors(Node x) { - whileActive: - do { - Node next = x.next; - // assert next != null; - // assert x != NEXT_TERMINATOR; - // assert x != PREV_TERMINATOR; - Node p = next; - findActive: - for (;;) { - if (p.item != null) - break findActive; - Node q = p.next; - if (q == null) { - if (p.prev == p) - continue whileActive; - break findActive; - } - else if (p == q) - continue whileActive; - else - p = q; - } - - // found active CAS target - if (next == p || x.casNext(next, p)) - return; - - } while (x.item != null || x.prev == null); - } - - /** - * Returns the successor of p, or the first node if p.next has been - * linked to self, which will only be true if traversing with a - * stale pointer that is now off the list. - */ - final Node succ(Node p) { - // TODO: should we skip deleted nodes here? - Node q = p.next; - return (p == q) ? first() : q; - } - - /** - * Returns the predecessor of p, or the last node if p.prev has been - * linked to self, which will only be true if traversing with a - * stale pointer that is now off the list. - */ - final Node pred(Node p) { - Node q = p.prev; - return (p == q) ? last() : q; - } - - /** - * Returns the first node, the unique node p for which: - * p.prev == null && p.next != p - * The returned node may or may not be logically deleted. - * Guarantees that head is set to the returned node. - */ - Node first() { - restartFromHead: - for (;;) - for (Node h = head, p = h, q;;) { - if ((q = p.prev) != null && - (q = (p = q).prev) != null) - // Check for head updates every other hop. - // If p == q, we are sure to follow head instead. - p = (h != (h = head)) ? h : q; - else if (p == h - // It is possible that p is PREV_TERMINATOR, - // but if so, the CAS is guaranteed to fail. - || casHead(h, p)) - return p; - else - continue restartFromHead; - } - } - - /** - * Returns the last node, the unique node p for which: - * p.next == null && p.prev != p - * The returned node may or may not be logically deleted. - * Guarantees that tail is set to the returned node. - */ - Node last() { - restartFromTail: - for (;;) - for (Node t = tail, p = t, q;;) { - if ((q = p.next) != null && - (q = (p = q).next) != null) - // Check for tail updates every other hop. - // If p == q, we are sure to follow tail instead. - p = (t != (t = tail)) ? t : q; - else if (p == t - // It is possible that p is NEXT_TERMINATOR, - // but if so, the CAS is guaranteed to fail. - || casTail(t, p)) - return p; - else - continue restartFromTail; - } - } - - // Minor convenience utilities - - /** - * Throws NullPointerException if argument is null. - * - * @param v the element - */ - private static void checkNotNull(Object v) { - if (v == null) - throw new NullPointerException(); - } - - /** - * Returns element unless it is null, in which case throws - * NoSuchElementException. - * - * @param v the element - * @return the element - */ - private E screenNullResult(E v) { - if (v == null) - throw new NoSuchElementException(); - return v; - } - - /** - * Creates an array list and fills it with elements of this list. - * Used by toArray. - * - * @return the array list - */ - private ArrayList toArrayList() { - ArrayList list = new ArrayList(); - for (Node p = first(); p != null; p = succ(p)) { - E item = p.item; - if (item != null) - list.add(item); - } - return list; - } - - /** - * Constructs an empty deque. - */ - public ConcurrentLinkedDeque() { - head = tail = new Node(null); - } - - /** - * Constructs a deque initially containing the elements of - * the given collection, added in traversal order of the - * collection's iterator. - * - * @param c the collection of elements to initially contain - * @throws NullPointerException if the specified collection or any - * of its elements are null - */ - public ConcurrentLinkedDeque(Collection c) { - // Copy c into a private chain of Nodes - Node h = null, t = null; - for (E e : c) { - checkNotNull(e); - Node newNode = new Node(e); - if (h == null) - h = t = newNode; - else { - t.lazySetNext(newNode); - newNode.lazySetPrev(t); - t = newNode; - } - } - initHeadTail(h, t); - } - - /** - * Initializes head and tail, ensuring invariants hold. - */ - private void initHeadTail(Node h, Node t) { - if (h == t) { - if (h == null) - h = t = new Node(null); - else { - // Avoid edge case of a single Node with non-null item. - Node newNode = new Node(null); - t.lazySetNext(newNode); - newNode.lazySetPrev(t); - t = newNode; - } - } - head = h; - tail = t; - } - - /** - * Inserts the specified element at the front of this deque. - * As the deque is unbounded, this method will never throw - * {@link IllegalStateException}. - * - * @throws NullPointerException if the specified element is null - */ - public void addFirst(E e) { - linkFirst(e); - } - - /** - * Inserts the specified element at the end of this deque. - * As the deque is unbounded, this method will never throw - * {@link IllegalStateException}. - * - *

This method is equivalent to {@link #add}. - * - * @throws NullPointerException if the specified element is null - */ - public void addLast(E e) { - linkLast(e); - } - - /** - * Inserts the specified element at the front of this deque. - * As the deque is unbounded, this method will never return {@code false}. - * - * @return {@code true} (as specified by {@link Deque#offerFirst}) - * @throws NullPointerException if the specified element is null - */ - public boolean offerFirst(E e) { - linkFirst(e); - return true; - } - - /** - * Inserts the specified element at the end of this deque. - * As the deque is unbounded, this method will never return {@code false}. - * - *

This method is equivalent to {@link #add}. - * - * @return {@code true} (as specified by {@link Deque#offerLast}) - * @throws NullPointerException if the specified element is null - */ - public boolean offerLast(E e) { - linkLast(e); - return true; - } - - public E peekFirst() { - for (Node p = first(); p != null; p = succ(p)) { - E item = p.item; - if (item != null) - return item; - } - return null; - } - - public E peekLast() { - for (Node p = last(); p != null; p = pred(p)) { - E item = p.item; - if (item != null) - return item; - } - return null; - } - - /** - * @throws NoSuchElementException {@inheritDoc} - */ - public E getFirst() { - return screenNullResult(peekFirst()); - } - - /** - * @throws NoSuchElementException {@inheritDoc} - */ - public E getLast() { - return screenNullResult(peekLast()); - } - - public E pollFirst() { - for (Node p = first(); p != null; p = succ(p)) { - E item = p.item; - if (item != null && p.casItem(item, null)) { - unlink(p); - return item; - } - } - return null; - } - - public E pollLast() { - for (Node p = last(); p != null; p = pred(p)) { - E item = p.item; - if (item != null && p.casItem(item, null)) { - unlink(p); - return item; - } - } - return null; - } - - /** - * @throws NoSuchElementException {@inheritDoc} - */ - public E removeFirst() { - return screenNullResult(pollFirst()); - } - - /** - * @throws NoSuchElementException {@inheritDoc} - */ - public E removeLast() { - return screenNullResult(pollLast()); - } - - // *** Queue and stack methods *** - - /** - * Inserts the specified element at the tail of this deque. - * As the deque is unbounded, this method will never return {@code false}. - * - * @return {@code true} (as specified by {@link Queue#offer}) - * @throws NullPointerException if the specified element is null - */ - public boolean offer(E e) { - return offerLast(e); - } - - /** - * Inserts the specified element at the tail of this deque. - * As the deque is unbounded, this method will never throw - * {@link IllegalStateException} or return {@code false}. - * - * @return {@code true} (as specified by {@link Collection#add}) - * @throws NullPointerException if the specified element is null - */ - public boolean add(E e) { - return offerLast(e); - } - - public E poll() { return pollFirst(); } - public E peek() { return peekFirst(); } - - /** - * @throws NoSuchElementException {@inheritDoc} - */ - public E remove() { return removeFirst(); } - - /** - * @throws NoSuchElementException {@inheritDoc} - */ - public E pop() { return removeFirst(); } - - /** - * @throws NoSuchElementException {@inheritDoc} - */ - public E element() { return getFirst(); } - - /** - * @throws NullPointerException {@inheritDoc} - */ - public void push(E e) { addFirst(e); } - - /** - * Removes the first element {@code e} such that - * {@code o.equals(e)}, if such an element exists in this deque. - * If the deque does not contain the element, it is unchanged. - * - * @param o element to be removed from this deque, if present - * @return {@code true} if the deque contained the specified element - * @throws NullPointerException if the specified element is null - */ - public boolean removeFirstOccurrence(Object o) { - checkNotNull(o); - for (Node p = first(); p != null; p = succ(p)) { - E item = p.item; - if (item != null && o.equals(item) && p.casItem(item, null)) { - unlink(p); - return true; - } - } - return false; - } - - /** - * Removes the last element {@code e} such that - * {@code o.equals(e)}, if such an element exists in this deque. - * If the deque does not contain the element, it is unchanged. - * - * @param o element to be removed from this deque, if present - * @return {@code true} if the deque contained the specified element - * @throws NullPointerException if the specified element is null - */ - public boolean removeLastOccurrence(Object o) { - checkNotNull(o); - for (Node p = last(); p != null; p = pred(p)) { - E item = p.item; - if (item != null && o.equals(item) && p.casItem(item, null)) { - unlink(p); - return true; - } - } - return false; - } - - /** - * Returns {@code true} if this deque contains at least one - * element {@code e} such that {@code o.equals(e)}. - * - * @param o element whose presence in this deque is to be tested - * @return {@code true} if this deque contains the specified element - */ - public boolean contains(Object o) { - if (o == null) return false; - for (Node p = first(); p != null; p = succ(p)) { - E item = p.item; - if (item != null && o.equals(item)) - return true; - } - return false; - } - - /** - * Returns {@code true} if this collection contains no elements. - * - * @return {@code true} if this collection contains no elements - */ - public boolean isEmpty() { - return peekFirst() == null; - } - - /** - * Returns the number of elements in this deque. If this deque - * contains more than {@code Integer.MAX_VALUE} elements, it - * returns {@code Integer.MAX_VALUE}. - * - *

Beware that, unlike in most collections, this method is - * NOT a constant-time operation. Because of the - * asynchronous nature of these deques, determining the current - * number of elements requires traversing them all to count them. - * Additionally, it is possible for the size to change during - * execution of this method, in which case the returned result - * will be inaccurate. Thus, this method is typically not very - * useful in concurrent applications. - * - * @return the number of elements in this deque - */ - public int size() { - int count = 0; - for (Node p = first(); p != null; p = succ(p)) - if (p.item != null) - // Collection.size() spec says to max out - if (++count == Integer.MAX_VALUE) - break; - return count; - } - - /** - * Removes the first element {@code e} such that - * {@code o.equals(e)}, if such an element exists in this deque. - * If the deque does not contain the element, it is unchanged. - * - * @param o element to be removed from this deque, if present - * @return {@code true} if the deque contained the specified element - * @throws NullPointerException if the specified element is null - */ - public boolean remove(Object o) { - return removeFirstOccurrence(o); - } - - /** - * Appends all of the elements in the specified collection to the end of - * this deque, in the order that they are returned by the specified - * collection's iterator. Attempts to {@code addAll} of a deque to - * itself result in {@code IllegalArgumentException}. - * - * @param c the elements to be inserted into this deque - * @return {@code true} if this deque changed as a result of the call - * @throws NullPointerException if the specified collection or any - * of its elements are null - * @throws IllegalArgumentException if the collection is this deque - */ - public boolean addAll(Collection c) { - if (c == this) - // As historically specified in AbstractQueue#addAll - throw new IllegalArgumentException(); - - // Copy c into a private chain of Nodes - Node beginningOfTheEnd = null, last = null; - for (E e : c) { - checkNotNull(e); - Node newNode = new Node(e); - if (beginningOfTheEnd == null) - beginningOfTheEnd = last = newNode; - else { - last.lazySetNext(newNode); - newNode.lazySetPrev(last); - last = newNode; - } - } - if (beginningOfTheEnd == null) - return false; - - // Atomically append the chain at the tail of this collection - restartFromTail: - for (;;) - for (Node t = tail, p = t, q;;) { - if ((q = p.next) != null && - (q = (p = q).next) != null) - // Check for tail updates every other hop. - // If p == q, we are sure to follow tail instead. - p = (t != (t = tail)) ? t : q; - else if (p.prev == p) // NEXT_TERMINATOR - continue restartFromTail; - else { - // p is last node - beginningOfTheEnd.lazySetPrev(p); // CAS piggyback - if (p.casNext(null, beginningOfTheEnd)) { - // Successful CAS is the linearization point - // for all elements to be added to this deque. - if (!casTail(t, last)) { - // Try a little harder to update tail, - // since we may be adding many elements. - t = tail; - if (last.next == null) - casTail(t, last); - } - return true; - } - // Lost CAS race to another thread; re-read next - } - } - } - - /** - * Removes all of the elements from this deque. - */ - public void clear() { - while (pollFirst() != null) - ; - } - - /** - * Returns an array containing all of the elements in this deque, in - * proper sequence (from first to last element). - * - *

The returned array will be "safe" in that no references to it are - * maintained by this deque. (In other words, this method must allocate - * a new array). The caller is thus free to modify the returned array. - * - *

This method acts as bridge between array-based and collection-based - * APIs. - * - * @return an array containing all of the elements in this deque - */ - public Object[] toArray() { - return toArrayList().toArray(); - } - - /** - * Returns an array containing all of the elements in this deque, - * in proper sequence (from first to last element); the runtime - * type of the returned array is that of the specified array. If - * the deque fits in the specified array, it is returned therein. - * Otherwise, a new array is allocated with the runtime type of - * the specified array and the size of this deque. - * - *

If this deque fits in the specified array with room to spare - * (i.e., the array has more elements than this deque), the element in - * the array immediately following the end of the deque is set to - * {@code null}. - * - *

Like the {@link #toArray()} method, this method acts as - * bridge between array-based and collection-based APIs. Further, - * this method allows precise control over the runtime type of the - * output array, and may, under certain circumstances, be used to - * save allocation costs. - * - *

Suppose {@code x} is a deque known to contain only strings. - * The following code can be used to dump the deque into a newly - * allocated array of {@code String}: - * - *

 {@code String[] y = x.toArray(new String[0]);}
- * - * Note that {@code toArray(new Object[0])} is identical in function to - * {@code toArray()}. - * - * @param a the array into which the elements of the deque are to - * be stored, if it is big enough; otherwise, a new array of the - * same runtime type is allocated for this purpose - * @return an array containing all of the elements in this deque - * @throws ArrayStoreException if the runtime type of the specified array - * is not a supertype of the runtime type of every element in - * this deque - * @throws NullPointerException if the specified array is null - */ - public T[] toArray(T[] a) { - return toArrayList().toArray(a); - } - - /** - * Returns an iterator over the elements in this deque in proper sequence. - * The elements will be returned in order from first (head) to last (tail). - * - *

The returned iterator is - * weakly consistent. - * - * @return an iterator over the elements in this deque in proper sequence - */ - public Iterator iterator() { - return new Itr(); - } - - /** - * Returns an iterator over the elements in this deque in reverse - * sequential order. The elements will be returned in order from - * last (tail) to first (head). - * - *

The returned iterator is - * weakly consistent. - * - * @return an iterator over the elements in this deque in reverse order - */ - public Iterator descendingIterator() { - return new DescendingItr(); - } - - private abstract class AbstractItr implements Iterator { - /** - * Next node to return item for. - */ - private Node nextNode; - - /** - * nextItem holds on to item fields because once we claim - * that an element exists in hasNext(), we must return it in - * the following next() call even if it was in the process of - * being removed when hasNext() was called. - */ - private E nextItem; - - /** - * Node returned by most recent call to next. Needed by remove. - * Reset to null if this element is deleted by a call to remove. - */ - private Node lastRet; - - abstract Node startNode(); - abstract Node nextNode(Node p); - - AbstractItr() { - advance(); - } - - /** - * Sets nextNode and nextItem to next valid node, or to null - * if no such. - */ - private void advance() { - lastRet = nextNode; - - Node p = (nextNode == null) ? startNode() : nextNode(nextNode); - for (;; p = nextNode(p)) { - if (p == null) { - // p might be active end or TERMINATOR node; both are OK - nextNode = null; - nextItem = null; - break; - } - E item = p.item; - if (item != null) { - nextNode = p; - nextItem = item; - break; - } - } - } - - public boolean hasNext() { - return nextItem != null; - } - - public E next() { - E item = nextItem; - if (item == null) throw new NoSuchElementException(); - advance(); - return item; - } - - public void remove() { - Node l = lastRet; - if (l == null) throw new IllegalStateException(); - l.item = null; - unlink(l); - lastRet = null; - } - } - - /** Forward iterator */ - private class Itr extends AbstractItr { - Node startNode() { return first(); } - Node nextNode(Node p) { return succ(p); } - } - - /** Descending iterator */ - private class DescendingItr extends AbstractItr { - Node startNode() { return last(); } - Node nextNode(Node p) { return pred(p); } - } - - /** A customized variant of Spliterators.IteratorSpliterator */ - static final class CLDSpliterator implements Spliterator { - static final int MAX_BATCH = 1 << 25; // max batch array size; - final ConcurrentLinkedDeque queue; - Node current; // current node; null until initialized - int batch; // batch size for splits - boolean exhausted; // true when no more nodes - CLDSpliterator(ConcurrentLinkedDeque queue) { - this.queue = queue; - } - - public Spliterator trySplit() { - Node p; - final ConcurrentLinkedDeque q = this.queue; - int b = batch; - int n = (b <= 0) ? 1 : (b >= MAX_BATCH) ? MAX_BATCH : b + 1; - if (!exhausted && - ((p = current) != null || (p = q.first()) != null)) { - if (p.item == null && p == (p = p.next)) - current = p = q.first(); - if (p != null && p.next != null) { - Object[] a = new Object[n]; - int i = 0; - do { - if ((a[i] = p.item) != null) - ++i; - if (p == (p = p.next)) - p = q.first(); - } while (p != null && i < n); - if ((current = p) == null) - exhausted = true; - if (i > 0) { - batch = i; - return Spliterators.spliterator - (a, 0, i, Spliterator.ORDERED | Spliterator.NONNULL | - Spliterator.CONCURRENT); - } - } - } - return null; - } - - public void forEachRemaining(Consumer action) { - Node p; - if (action == null) throw new NullPointerException(); - final ConcurrentLinkedDeque q = this.queue; - if (!exhausted && - ((p = current) != null || (p = q.first()) != null)) { - exhausted = true; - do { - E e = p.item; - if (p == (p = p.next)) - p = q.first(); - if (e != null) - action.accept(e); - } while (p != null); - } - } - - public boolean tryAdvance(Consumer action) { - Node p; - if (action == null) throw new NullPointerException(); - final ConcurrentLinkedDeque q = this.queue; - if (!exhausted && - ((p = current) != null || (p = q.first()) != null)) { - E e; - do { - e = p.item; - if (p == (p = p.next)) - p = q.first(); - } while (e == null && p != null); - if ((current = p) == null) - exhausted = true; - if (e != null) { - action.accept(e); - return true; - } - } - return false; - } - - public long estimateSize() { return Long.MAX_VALUE; } - - public int characteristics() { - return Spliterator.ORDERED | Spliterator.NONNULL | - Spliterator.CONCURRENT; - } - } - - /** - * Returns a {@link Spliterator} over the elements in this deque. - * - *

The returned spliterator is - * weakly consistent. - * - *

The {@code Spliterator} reports {@link Spliterator#CONCURRENT}, - * {@link Spliterator#ORDERED}, and {@link Spliterator#NONNULL}. - * - * @implNote - * The {@code Spliterator} implements {@code trySplit} to permit limited - * parallelism. - * - * @return a {@code Spliterator} over the elements in this deque - * @since 1.8 - */ - public Spliterator spliterator() { - return new CLDSpliterator(this); - } - - /** - * Saves this deque to a stream (that is, serializes it). - * - * @param s the stream - * @throws java.io.IOException if an I/O error occurs - * @serialData All of the elements (each an {@code E}) in - * the proper order, followed by a null - */ - private void writeObject(java.io.ObjectOutputStream s) - throws java.io.IOException { - - // Write out any hidden stuff - s.defaultWriteObject(); - - // Write out all elements in the proper order. - for (Node p = first(); p != null; p = succ(p)) { - E item = p.item; - if (item != null) - s.writeObject(item); - } - - // Use trailing null as sentinel - s.writeObject(null); - } - - /** - * Reconstitutes this deque from a stream (that is, deserializes it). - * @param s the stream - * @throws ClassNotFoundException if the class of a serialized object - * could not be found - * @throws java.io.IOException if an I/O error occurs - */ - private void readObject(java.io.ObjectInputStream s) - throws java.io.IOException, ClassNotFoundException { - s.defaultReadObject(); - - // Read in elements until trailing null sentinel found - Node h = null, t = null; - Object item; - while ((item = s.readObject()) != null) { - @SuppressWarnings("unchecked") - Node newNode = new Node((E) item); - if (h == null) - h = t = newNode; - else { - t.lazySetNext(newNode); - newNode.lazySetPrev(t); - t = newNode; - } - } - initHeadTail(h, t); - } - - private boolean casHead(Node cmp, Node val) { - return UNSAFE.compareAndSwapObject(this, headOffset, cmp, val); - } - - private boolean casTail(Node cmp, Node val) { - return UNSAFE.compareAndSwapObject(this, tailOffset, cmp, val); - } - - // Unsafe mechanics - - private static final sun.misc.Unsafe UNSAFE; - private static final long headOffset; - private static final long tailOffset; - static { - PREV_TERMINATOR = new Node(); - PREV_TERMINATOR.next = PREV_TERMINATOR; - NEXT_TERMINATOR = new Node(); - NEXT_TERMINATOR.prev = NEXT_TERMINATOR; - try { - UNSAFE = sun.misc.Unsafe.getUnsafe(); - Class k = ConcurrentLinkedDeque.class; - headOffset = UNSAFE.objectFieldOffset - (k.getDeclaredField("head")); - tailOffset = UNSAFE.objectFieldOffset - (k.getDeclaredField("tail")); - } catch (Exception e) { - throw new Error(e); - } - } -} --- /dev/null 2015-06-06 09:45:45.596162458 +0200 +++ new/src/java.base/share/classes/java/lang/ref/FinalizerList.java 2015-06-06 15:34:11.819911449 +0200 @@ -0,0 +1,559 @@ +/* + * 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. Oracle designates this + * particular file as subject to the "Classpath" exception as provided + * by Oracle in the LICENSE file that accompanied this code. + * + * This code is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License + * version 2 for more details (a copy is included in the LICENSE file that + * accompanied this code). + * + * You should have received a copy of the GNU General Public License version + * 2 along with this work; if not, write to the Free Software Foundation, + * Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA. + * + * Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA + * or visit www.oracle.com if you need additional information or have any + * questions. + */ + +/* + * This file is available under and governed by the GNU General Public + * License version 2 only, as published by the Free Software Foundation. + * However, the following notice accompanied the original version of this + * file: + * + * Written by Doug Lea and Martin Buchholz with assistance from members of + * JCP JSR-166 Expert Group and released to the public domain, as explained + * at http://creativecommons.org/publicdomain/zero/1.0/ + */ + +package java.lang.ref; + +/** + * A concurrent doubly-linked list of {@link java.lang.ref.Finalizer} nodes + * modeled by {@link java.util.concurrent.ConcurrentLinkedDeque}. + */ +final class FinalizerList { + + /** + * A node from which the first node on list (that is, the unique node p + * with p.prev == null && p.next != p) can be reached in O(1) time. + * Invariants: + * - the first node is always O(1) reachable from head via prev links + * - all live nodes are reachable from the first node via succ() + * - head != null + * - (tmp = head).next != tmp || tmp != head + * - head is never gc-unlinked (but may be unlinked) + * Non-invariants: + * - head.item may or may not be null + * - head may not be reachable from the first or last node, or from tail + */ + private volatile Finalizer head; + + /** + * A node from which the last node on list (that is, the unique node p + * with p.next == null && p.prev != p) can be reached in O(1) time. + * Invariants: + * - the last node is always O(1) reachable from tail via next links + * - all live nodes are reachable from the last node via pred() + * - tail != null + * - tail is never gc-unlinked (but may be unlinked) + * Non-invariants: + * - tail.item may or may not be null + * - tail may not be reachable from the first or last node, or from head + */ + private volatile Finalizer tail; + + private static final Finalizer PREV_TERMINATOR, NEXT_TERMINATOR; + + /** + * Links newFinalizer as first or last element, depending on + * specified boolean flag. + */ + void link(Finalizer newFinalizer, boolean first) { + if (first) { + linkFirst(newFinalizer); + } else { + linkLast(newFinalizer); + } + } + + /** + * Links newFinalizer as first element. + */ + private void linkFirst(Finalizer newFinalizer) { + + restartFromHead: + for (;;) + for (Finalizer h = head, p = h, q;;) { + if ((q = p.prev) != null && + (q = (p = q).prev) != null) + // Check for head updates every other hop. + // If p == q, we are sure to follow head instead. + p = (h != (h = head)) ? h : q; + else if (p.next == p) // PREV_TERMINATOR + continue restartFromHead; + else { + // p is first node + newFinalizer.lazySetNext(p); // CAS piggyback + if (p.casPrev(null, newFinalizer)) { + // Successful CAS is the linearization point + // for e to become an element of this deque, + // and for newNode to become "live". + if (p != h) // hop two nodes at a time + casHead(h, newFinalizer); // Failure is OK. + return; + } + // Lost CAS race to another thread; re-read prev + } + } + } + + /** + * Links newNode as last element. + */ + private void linkLast(Finalizer newFinalizer) { + + restartFromTail: + for (;;) + for (Finalizer t = tail, p = t, q;;) { + if ((q = p.next) != null && + (q = (p = q).next) != null) + // Check for tail updates every other hop. + // If p == q, we are sure to follow tail instead. + p = (t != (t = tail)) ? t : q; + else if (p.prev == p) // NEXT_TERMINATOR + continue restartFromTail; + else { + // p is last node + newFinalizer.lazySetPrev(p); // CAS piggyback + if (p.casNext(null, newFinalizer)) { + // Successful CAS is the linearization point + // for e to become an element of this deque, + // and for newNode to become "live". + if (p != t) // hop two nodes at a time + casTail(t, newFinalizer); // Failure is OK. + return; + } + // Lost CAS race to another thread; re-read next + } + } + } + + private static final int HOPS = 2; + + /** + * Unlinks non-null node x. + */ + void unlink(Finalizer x) { + // assert x != null; + // assert x.item == null; + // assert x != PREV_TERMINATOR; + // assert x != NEXT_TERMINATOR; + + final Finalizer prev = x.prev; + final Finalizer next = x.next; + if (prev == null) { + unlinkFirst(x, next); + } else if (next == null) { + unlinkLast(x, prev); + } else { + // Unlink interior node. + // + // This is the common case, since a series of polls at the + // same end will be "interior" removes, except perhaps for + // the first one, since end nodes cannot be unlinked. + // + // At any time, all active nodes are mutually reachable by + // following a sequence of either next or prev pointers. + // + // Our strategy is to find the unique active predecessor + // and successor of x. Try to fix up their links so that + // they point to each other, leaving x unreachable from + // active nodes. If successful, and if x has no live + // predecessor/successor, we additionally try to gc-unlink, + // leaving active nodes unreachable from x, by rechecking + // that the status of predecessor and successor are + // unchanged and ensuring that x is not reachable from + // tail/head, before setting x's prev/next links to their + // logical approximate replacements, self/TERMINATOR. + Finalizer activePred, activeSucc; + boolean isFirst, isLast; + int hops = 1; + + // Find active predecessor + for (Finalizer p = prev; ; ++hops) { + if (p.isAlive()) { + activePred = p; + isFirst = false; + break; + } + Finalizer q = p.prev; + if (q == null) { + if (p.next == p) + return; + activePred = p; + isFirst = true; + break; + } + else if (p == q) + return; + else + p = q; + } + + // Find active successor + for (Finalizer p = next; ; ++hops) { + if (p.isAlive()) { + activeSucc = p; + isLast = false; + break; + } + Finalizer q = p.next; + if (q == null) { + if (p.prev == p) + return; + activeSucc = p; + isLast = true; + break; + } + else if (p == q) + return; + else + p = q; + } + + // TODO: better HOP heuristics + if (hops < HOPS + // always squeeze out interior deleted nodes + && (isFirst | isLast)) + return; + + // Squeeze out deleted nodes between activePred and + // activeSucc, including x. + skipDeletedSuccessors(activePred); + skipDeletedPredecessors(activeSucc); + + // Try to gc-unlink, if possible + if ((isFirst | isLast) && + + // Recheck expected state of predecessor and successor + (activePred.next == activeSucc) && + (activeSucc.prev == activePred) && + (isFirst ? activePred.prev == null : activePred.isAlive()) && + (isLast ? activeSucc.next == null : activeSucc.isAlive())) { + + updateHead(); // Ensure x is not reachable from head + updateTail(); // Ensure x is not reachable from tail + + // Finally, actually gc-unlink + x.lazySetPrev(isFirst ? PREV_TERMINATOR : x); + x.lazySetNext(isLast ? NEXT_TERMINATOR : x); + } + } + } + + /** + * Unlinks non-null first node. + */ + private void unlinkFirst(Finalizer first, Finalizer next) { + // assert first != null; + // assert next != null; + // assert first.item == null; + for (Finalizer o = null, p = next, q;;) { + if (p.isAlive() || (q = p.next) == null) { + if (o != null && p.prev != p && first.casNext(next, p)) { + skipDeletedPredecessors(p); + if (first.prev == null && + (p.next == null || p.isAlive()) && + p.prev == first) { + + updateHead(); // Ensure o is not reachable from head + updateTail(); // Ensure o is not reachable from tail + + // Finally, actually gc-unlink + o.lazySetNext(o); + o.lazySetPrev(PREV_TERMINATOR); + } + } + return; + } + else if (p == q) + return; + else { + o = p; + p = q; + } + } + } + + /** + * Unlinks non-null last node. + */ + private void unlinkLast(Finalizer last, Finalizer prev) { + // assert last != null; + // assert prev != null; + // assert last.item == null; + for (Finalizer o = null, p = prev, q;;) { + if (p.isAlive() || (q = p.prev) == null) { + if (o != null && p.next != p && last.casPrev(prev, p)) { + skipDeletedSuccessors(p); + if (last.next == null && + (p.prev == null || p.isAlive()) && + p.next == last) { + + updateHead(); // Ensure o is not reachable from head + updateTail(); // Ensure o is not reachable from tail + + // Finally, actually gc-unlink + o.lazySetPrev(o); + o.lazySetNext(NEXT_TERMINATOR); + } + } + return; + } + else if (p == q) + return; + else { + o = p; + p = q; + } + } + } + + /** + * Guarantees that any node which was unlinked before a call to + * this method will be unreachable from head after it returns. + * Does not guarantee to eliminate slack, only that head will + * point to a node that was active while this method was running. + */ + private void updateHead() { + // Either head already points to an active node, or we keep + // trying to cas it to the first node until it does. + Finalizer h, p, q; + restartFromHead: + while ((h = head).isDeleted() && (p = h.prev) != null) { + for (;;) { + if ((q = p.prev) == null || + (q = (p = q).prev) == null) { + // It is possible that p is PREV_TERMINATOR, + // but if so, the CAS is guaranteed to fail. + if (casHead(h, p)) + return; + else + continue restartFromHead; + } + else if (h != head) + continue restartFromHead; + else + p = q; + } + } + } + + /** + * Guarantees that any node which was unlinked before a call to + * this method will be unreachable from tail after it returns. + * Does not guarantee to eliminate slack, only that tail will + * point to a node that was active while this method was running. + */ + private void updateTail() { + // Either tail already points to an active node, or we keep + // trying to cas it to the last node until it does. + Finalizer t, p, q; + restartFromTail: + while ((t = tail).isDeleted() && (p = t.next) != null) { + for (;;) { + if ((q = p.next) == null || + (q = (p = q).next) == null) { + // It is possible that p is NEXT_TERMINATOR, + // but if so, the CAS is guaranteed to fail. + if (casTail(t, p)) + return; + else + continue restartFromTail; + } + else if (t != tail) + continue restartFromTail; + else + p = q; + } + } + } + + private void skipDeletedPredecessors(Finalizer x) { + whileActive: + do { + Finalizer prev = x.prev; + // assert prev != null; + // assert x != NEXT_TERMINATOR; + // assert x != PREV_TERMINATOR; + Finalizer p = prev; + findActive: + for (;;) { + if (p.isAlive()) + break findActive; + Finalizer q = p.prev; + if (q == null) { + if (p.next == p) + continue whileActive; + break findActive; + } + else if (p == q) + continue whileActive; + else + p = q; + } + + // found active CAS target + if (prev == p || x.casPrev(prev, p)) + return; + + } while (x.isAlive() || x.next == null); + } + + private void skipDeletedSuccessors(Finalizer x) { + whileActive: + do { + Finalizer next = x.next; + // assert next != null; + // assert x != NEXT_TERMINATOR; + // assert x != PREV_TERMINATOR; + Finalizer p = next; + findActive: + for (;;) { + if (p.isAlive()) + break findActive; + Finalizer q = p.next; + if (q == null) { + if (p.prev == p) + continue whileActive; + break findActive; + } + else if (p == q) + continue whileActive; + else + p = q; + } + + // found active CAS target + if (next == p || x.casNext(next, p)) + return; + + } while (x.isAlive() || x.prev == null); + } + + /** + * Returns the successor of p, or the first node if p.next has been + * linked to self, which will only be true if traversing with a + * stale pointer that is now off the list. + */ + Finalizer succ(Finalizer p) { + // TODO: should we skip deleted nodes here? + Finalizer q = p.next; + return (p == q) ? first() : q; + } + + /** + * Returns the predecessor of p, or the last node if p.prev has been + * linked to self, which will only be true if traversing with a + * stale pointer that is now off the list. + */ + Finalizer pred(Finalizer p) { + Finalizer q = p.prev; + return (p == q) ? last() : q; + } + + /** + * Returns the first node, the unique node p for which: + * p.prev == null && p.next != p + * The returned node may or may not be logically deleted. + * Guarantees that head is set to the returned node. + */ + Finalizer first() { + restartFromHead: + for (;;) + for (Finalizer h = head, p = h, q;;) { + if ((q = p.prev) != null && + (q = (p = q).prev) != null) + // Check for head updates every other hop. + // If p == q, we are sure to follow head instead. + p = (h != (h = head)) ? h : q; + else if (p == h + // It is possible that p is PREV_TERMINATOR, + // but if so, the CAS is guaranteed to fail. + || casHead(h, p)) + return p; + else + continue restartFromHead; + } + } + + /** + * Returns the last node, the unique node p for which: + * p.next == null && p.prev != p + * The returned node may or may not be logically deleted. + * Guarantees that tail is set to the returned node. + */ + Finalizer last() { + restartFromTail: + for (;;) + for (Finalizer t = tail, p = t, q;;) { + if ((q = p.next) != null && + (q = (p = q).next) != null) + // Check for tail updates every other hop. + // If p == q, we are sure to follow tail instead. + p = (t != (t = tail)) ? t : q; + else if (p == t + // It is possible that p is NEXT_TERMINATOR, + // but if so, the CAS is guaranteed to fail. + || casTail(t, p)) + return p; + else + continue restartFromTail; + } + } + + /** + * Constructs an empty list. + */ + FinalizerList() { + head = tail = new Finalizer(); + } + + // Unsafe mechanics + + private boolean casHead(Finalizer cmp, Finalizer val) { + return UNSAFE.compareAndSwapObject(this, HEAD, cmp, val); + } + + private boolean casTail(Finalizer cmp, Finalizer val) { + return UNSAFE.compareAndSwapObject(this, TAIL, cmp, val); + } + + private static final sun.misc.Unsafe UNSAFE; + private static final long HEAD; + private static final long TAIL; + static { + PREV_TERMINATOR = new Finalizer(); + PREV_TERMINATOR.next = PREV_TERMINATOR; + NEXT_TERMINATOR = new Finalizer(); + NEXT_TERMINATOR.prev = NEXT_TERMINATOR; + try { + UNSAFE = sun.misc.Unsafe.getUnsafe(); + Class flc = FinalizerList.class; + HEAD = UNSAFE.objectFieldOffset + (flc.getDeclaredField("head")); + TAIL = UNSAFE.objectFieldOffset + (flc.getDeclaredField("tail")); + } catch (Exception e) { + throw new Error(e); + } + } +} --- /dev/null 2015-06-06 09:45:45.596162458 +0200 +++ new/src/java.base/share/classes/java/lang/ref/ReferenceHandling.java 2015-06-06 15:34:12.115906266 +0200 @@ -0,0 +1,125 @@ +package java.lang.ref; + +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.ForkJoinWorkerThread; +import java.util.concurrent.RecursiveAction; +import java.util.concurrent.TimeUnit; + +/** + * A holder for a ForkJoinPool and tasks executing Finalizer(s), Cleaner(s) and + * enqueuing other Reference(s). + */ +final class ReferenceHandling { + + private static final ForkJoinPool pool = createPool(); + + /** + * Starts handling of references. Called from {@code Reference.} + * after VM has booted. + */ + static void start() { + // triggers class initialization if not already initialized + pool.getClass(); + } + + /** + * Called from {@link Finalizer#runFinalization()} as part of forked secondary + * finalizer thread to run all pending finalizers. We just help the ForkJoinPool + * by waiting for it to quiesce. + */ + static void runFinalization() { + do { + } while (!pool.awaitQuiescence(5, TimeUnit.SECONDS)); + } + + /** + * Creates new ForkJoinPool for handling the references. + */ + private static ForkJoinPool createPool() { + String refHandlingThreadsString = + System.getProperty("java.lang.ref.referenceHandlingThreads", "1"); + int referenceHandlingThreads; + try { + referenceHandlingThreads = Math.min( + Runtime.getRuntime().availableProcessors(), + Math.max(1, Integer.parseInt(refHandlingThreadsString)) + ); + } catch (NumberFormatException e) { + referenceHandlingThreads = 1; + } + return new ForkJoinPool( + referenceHandlingThreads, + ForkJoinPool.defaultForkJoinWorkerThreadFactory, + null, + true + ); + } + + /** + * A task that handles one chunk of references. + */ + static final class PendingChunkHandler extends RecursiveAction { + private Reference chunk; + + PendingChunkHandler(Reference chunk) { + this.chunk = chunk; + } + + @Override + protected void compute() { + Reference r = this.chunk; + if (r != null) { + this.chunk = null; + Reference.handlePendingChunk(r); + } + } + + void submit() { + if (Thread.currentThread() instanceof ForkJoinWorkerThread) { + // internal submission + fork(); + } else { + // external submission + pool.submit(this); + } + } + } + + /** + * A task for handling a chunk of Finaliz(ato|e)r(s). + */ + static final class FinalizrHandler extends RecursiveAction { + private Reference finalzrs; + + FinalizrHandler(Reference finalzrs) { + this.finalzrs = finalzrs; + } + + @Override + protected void compute() { + Reference f = finalzrs; + if (f != null) { + finalzrs = null; + for (Reference n = f.next; ; f = n, n = f.next) { + f.next = f; + // Finalizer and Finalizator are both Runnable(s) + ((Runnable) f).run(); + if (n == f) { // last in chunk + break; + } + } + } + } + + void submit() { + if (Thread.currentThread() instanceof ForkJoinWorkerThread) { + // internal submission + fork(); + } else { + // external submission + pool.submit(this); + } + } + } +} +