Simplify code around JobSupport (#4079)

This change simplifies some code used in for `JobSupport`:
there are fewer wrapper classes, the information is passed
between functions with less duplication, and some API was either
removed due to being unused, or moved to reduce its visibility.
diff --git a/kotlinx-coroutines-core/common/src/Await.kt b/kotlinx-coroutines-core/common/src/Await.kt
index b8a76f1..5cdc94f 100644
--- a/kotlinx-coroutines-core/common/src/Await.kt
+++ b/kotlinx-coroutines-core/common/src/Await.kt
@@ -114,5 +114,7 @@
                 // Note that all deferreds are complete here, so we don't need to dispose their nodes
             }
         }
+
+        override val onCancelling = false
     }
 }
diff --git a/kotlinx-coroutines-core/common/src/CancellableContinuationImpl.kt b/kotlinx-coroutines-core/common/src/CancellableContinuationImpl.kt
index 10f62d5..a8fe503 100644
--- a/kotlinx-coroutines-core/common/src/CancellableContinuationImpl.kt
+++ b/kotlinx-coroutines-core/common/src/CancellableContinuationImpl.kt
@@ -234,13 +234,6 @@
         }
     }
 
-    private fun callCancelHandler(handler: InternalCompletionHandler, cause: Throwable?) =
-        /*
-        * :KLUDGE: We have to invoke a handler in platform-specific way via `invokeIt` extension,
-        * because we play type tricks on Kotlin/JS and handler is not necessarily a function there
-        */
-        callCancelHandlerSafely { handler.invoke(cause) }
-
     fun callCancelHandler(handler: CancelHandler, cause: Throwable?) =
         callCancelHandlerSafely { handler.invoke(cause) }
 
@@ -341,10 +334,7 @@
     private fun installParentHandle(): DisposableHandle? {
         val parent = context[Job] ?: return null // don't do anything without a parent
         // Install the handle
-        val handle = parent.invokeOnCompletion(
-            onCancelling = true,
-            handler = ChildContinuation(this)
-        )
+        val handle = parent.invokeOnCompletion(handler = ChildContinuation(this))
         _parentHandle.compareAndSet(null, handle)
         return handle
     }
@@ -627,8 +617,6 @@
  * as seen from the debugger.
  * Use [UserSupplied] to create an instance from a lambda.
  * We can't avoid defining a separate type, because on JS, you can't inherit from a function type.
- *
- * @see InternalCompletionHandler for a very similar interface, but used for handling completion and not cancellation.
  */
 internal interface CancelHandler : NotCompleted {
     /**
@@ -682,8 +670,10 @@
 // Same as ChildHandleNode, but for cancellable continuation
 private class ChildContinuation(
     @JvmField val child: CancellableContinuationImpl<*>
-) : JobCancellingNode() {
+) : JobNode() {
     override fun invoke(cause: Throwable?) {
         child.parentCancelled(child.getContinuationCancellationCause(job))
     }
+
+    override val onCancelling = true
 }
diff --git a/kotlinx-coroutines-core/common/src/CompletionHandler.common.kt b/kotlinx-coroutines-core/common/src/CompletionHandler.common.kt
index 0a0176e..ea56b15 100644
--- a/kotlinx-coroutines-core/common/src/CompletionHandler.common.kt
+++ b/kotlinx-coroutines-core/common/src/CompletionHandler.common.kt
@@ -25,47 +25,3 @@
  */
 // TODO: deprecate. This doesn't seem better than a simple function type.
 public typealias CompletionHandler = (cause: Throwable?) -> Unit
-
-/**
- * Essentially the same as just a function from `Throwable?` to `Unit`.
- * The only thing implementors can do is call [invoke].
- * The reason this abstraction exists is to allow providing a readable [toString] in the list of completion handlers
- * as seen from the debugger.
- * Use [UserSupplied] to create an instance from a lambda.
- * We can't avoid defining a separate type, because on JS, you can't inherit from a function type.
- *
- * @see CancelHandler for a very similar interface, but used for handling cancellation and not completion.
- */
-internal interface InternalCompletionHandler {
-    /**
-     * Signals completion.
-     *
-     * This function:
-     * - Does not throw any exceptions.
-     *   For [Job] instances that are coroutines, exceptions thrown by this function will be caught, wrapped into
-     *   [CompletionHandlerException], and passed to [handleCoroutineException], but for those that are not coroutines,
-     *   they will just be rethrown, potentially crashing unrelated code.
-     * - Is fast, non-blocking, and thread-safe.
-     * - Can be invoked concurrently with the surrounding code.
-     * - Can be invoked from any context.
-     *
-     * The meaning of `cause` that is passed to the handler is:
-     * - It is `null` if the job has completed normally.
-     * - It is an instance of [CancellationException] if the job was cancelled _normally_.
-     *   **It should not be treated as an error**. In particular, it should not be reported to error logs.
-     * - Otherwise, the job had _failed_.
-     */
-    fun invoke(cause: Throwable?)
-
-    /**
-     * A lambda passed from outside the coroutine machinery.
-     *
-     * See the requirements for [InternalCompletionHandler.invoke] when implementing this function.
-     */
-    class UserSupplied(private val handler: (cause: Throwable?) -> Unit) : InternalCompletionHandler {
-        /** @suppress */
-        override fun invoke(cause: Throwable?) { handler(cause) }
-
-        override fun toString() = "InternalCompletionHandler.UserSupplied[${handler.classSimpleName}@$hexAddress]"
-    }
-}
diff --git a/kotlinx-coroutines-core/common/src/Exceptions.common.kt b/kotlinx-coroutines-core/common/src/Exceptions.common.kt
index ed20d02..e19c36f 100644
--- a/kotlinx-coroutines-core/common/src/Exceptions.common.kt
+++ b/kotlinx-coroutines-core/common/src/Exceptions.common.kt
@@ -1,7 +1,7 @@
 package kotlinx.coroutines
 
 /**
- * This exception gets thrown if an exception is caught while processing [InternalCompletionHandler] invocation for [Job].
+ * This exception gets thrown if an exception is caught while processing [CompletionHandler] invocation for [Job].
  *
  * @suppress **This an internal API and should not be used from general code.**
  */
diff --git a/kotlinx-coroutines-core/common/src/Job.kt b/kotlinx-coroutines-core/common/src/Job.kt
index e9a935c..5460c74 100644
--- a/kotlinx-coroutines-core/common/src/Job.kt
+++ b/kotlinx-coroutines-core/common/src/Job.kt
@@ -340,10 +340,6 @@
  * If the handler would have been invoked earlier if it was registered at that time, then it is invoked immediately,
  * unless [invokeImmediately] is set to `false`.
  *
- * The handler is scheduled to be invoked once the job is cancelled or is complete.
- * This behavior can be changed by setting the [onCancelling] parameter to `true`.
- * In this case, the handler is invoked as soon as the job becomes _cancelling_ instead.
- *
  * The meaning of `cause` that is passed to the handler is:
  * - It is `null` if the job has completed normally.
  * - It is an instance of [CancellationException] if the job was cancelled _normally_.
@@ -356,12 +352,11 @@
  * all the handlers are released when this job completes.
  */
 internal fun Job.invokeOnCompletion(
-    onCancelling: Boolean = false,
     invokeImmediately: Boolean = true,
-    handler: InternalCompletionHandler
+    handler: JobNode,
 ): DisposableHandle = when (this) {
-    is JobSupport -> invokeOnCompletionInternal(onCancelling, invokeImmediately, handler)
-    else -> invokeOnCompletion(onCancelling, invokeImmediately, handler::invoke)
+    is JobSupport -> invokeOnCompletionInternal(invokeImmediately, handler)
+    else -> invokeOnCompletion(handler.onCancelling, invokeImmediately, handler::invoke)
 }
 
 /**
@@ -672,3 +667,11 @@
      */
     override fun toString(): String = "NonDisposableHandle"
 }
+
+private class DisposeOnCompletion(
+    private val handle: DisposableHandle
+) : JobNode() {
+    override fun invoke(cause: Throwable?) = handle.dispose()
+
+    override val onCancelling = false
+}
diff --git a/kotlinx-coroutines-core/common/src/JobSupport.kt b/kotlinx-coroutines-core/common/src/JobSupport.kt
index fa9235d..e5e981b 100644
--- a/kotlinx-coroutines-core/common/src/JobSupport.kt
+++ b/kotlinx-coroutines-core/common/src/JobSupport.kt
@@ -144,7 +144,6 @@
             return
         }
         parent.start() // make sure the parent is started
-        @Suppress("DEPRECATION")
         val handle = parent.attachChild(this)
         parentHandle = handle
         // now check our state _after_ registering (see tryFinalizeSimpleState order of actions)
@@ -325,7 +324,7 @@
     private fun notifyCancelling(list: NodeList, cause: Throwable) {
         // first cancel our own children
         onCancelling(cause)
-        notifyHandlers<JobCancellingNode>(list, cause)
+        notifyHandlers(list, cause) { it.onCancelling }
         // then cancel parent
         cancelParent(cause) // tentative cancellation -- does not matter if there is no parent
     }
@@ -357,16 +356,18 @@
     }
 
     private fun NodeList.notifyCompletion(cause: Throwable?) =
-        notifyHandlers<JobNode>(this, cause)
+        notifyHandlers(this, cause) { true }
 
-    private inline fun <reified T: JobNode> notifyHandlers(list: NodeList, cause: Throwable?) {
+    private inline fun notifyHandlers(list: NodeList, cause: Throwable?, predicate: (JobNode) -> Boolean) {
         var exception: Throwable? = null
-        list.forEach<T> { node ->
-            try {
-                node.invoke(cause)
-            } catch (ex: Throwable) {
-                exception?.apply { addSuppressed(ex) } ?: run {
-                    exception =  CompletionHandlerException("Exception in completion handler $node for $this", ex)
+        list.forEach { node ->
+            if (node is JobNode && predicate(node)) {
+                try {
+                    node.invoke(cause)
+                } catch (ex: Throwable) {
+                    exception?.apply { addSuppressed(ex) } ?: run {
+                        exception = CompletionHandlerException("Exception in completion handler $node for $this", ex)
+                    }
                 }
             }
         }
@@ -444,26 +445,25 @@
 
     public final override fun invokeOnCompletion(handler: CompletionHandler): DisposableHandle =
         invokeOnCompletionInternal(
-            onCancelling = false,
             invokeImmediately = true,
-            handler = InternalCompletionHandler.UserSupplied(handler)
+            node = InvokeOnCompletion(handler),
         )
 
     public final override fun invokeOnCompletion(onCancelling: Boolean, invokeImmediately: Boolean, handler: CompletionHandler): DisposableHandle =
         invokeOnCompletionInternal(
-            onCancelling = onCancelling,
             invokeImmediately = invokeImmediately,
-            handler = InternalCompletionHandler.UserSupplied(handler)
+            node = if (onCancelling) {
+                InvokeOnCancelling(handler)
+            } else {
+                InvokeOnCompletion(handler)
+            }
         )
 
     internal fun invokeOnCompletionInternal(
-        onCancelling: Boolean,
         invokeImmediately: Boolean,
-        handler: InternalCompletionHandler
+        node: JobNode
     ): DisposableHandle {
-        // Create node upfront -- for common cases it just initializes JobNode.job field,
-        // for user-defined handlers it allocates a JobNode object that we might not need, but this is Ok.
-        val node: JobNode = makeNode(handler, onCancelling)
+        node.job = this
         loopOnState { state ->
             when (state) {
                 is Empty -> { // EMPTY_X state -- no completion handlers
@@ -480,13 +480,13 @@
                     } else {
                         var rootCause: Throwable? = null
                         var handle: DisposableHandle = NonDisposableHandle
-                        if (onCancelling && state is Finishing) {
+                        if (node.onCancelling && state is Finishing) {
                             synchronized(state) {
                                 // check if we are installing cancellation handler on job that is being cancelled
                                 rootCause = state.rootCause // != null if cancelling job
                                 // We add node to the list in two cases --- either the job is not being cancelled
                                 // or we are adding a child to a coroutine that is not completing yet
-                                if (rootCause == null || handler is ChildHandleNode && !state.isCompleting) {
+                                if (rootCause == null || node is ChildHandleNode && !state.isCompleting) {
                                     // Note: add node the list while holding lock on state (make sure it cannot change)
                                     if (!addLastAtomic(state, list, node)) return@loopOnState // retry
                                     // just return node if we don't have to invoke handler (not cancelling yet)
@@ -498,7 +498,7 @@
                         }
                         if (rootCause != null) {
                             // Note: attachChild uses invokeImmediately, so it gets invoked when adding to cancelled job
-                            if (invokeImmediately) handler.invoke(rootCause)
+                            if (invokeImmediately) node.invoke(rootCause)
                             return handle
                         } else {
                             if (addLastAtomic(state, list, node)) return node
@@ -508,26 +508,13 @@
                 else -> { // is complete
                     // :KLUDGE: We have to invoke a handler in platform-specific way via `invokeIt` extension,
                     // because we play type tricks on Kotlin/JS and handler is not necessarily a function there
-                    if (invokeImmediately) handler.invoke((state as? CompletedExceptionally)?.cause)
+                    if (invokeImmediately) node.invoke((state as? CompletedExceptionally)?.cause)
                     return NonDisposableHandle
                 }
             }
         }
     }
 
-    private fun makeNode(handler: InternalCompletionHandler, onCancelling: Boolean): JobNode {
-        val node = if (onCancelling) {
-            (handler as? JobCancellingNode)
-                ?: InvokeOnCancelling(handler)
-        } else {
-            (handler as? JobNode)
-                ?.also { assert { it !is JobCancellingNode } }
-                ?: InvokeOnCompletion(handler)
-        }
-        node.job = this
-        return node
-    }
-
     private fun addLastAtomic(expect: Any, list: NodeList, node: JobNode) =
         list.addLastIf(node) { this.state === expect }
 
@@ -590,6 +577,7 @@
         override fun invoke(cause: Throwable?) {
             select.trySelect(this@JobSupport, Unit)
         }
+        override val onCancelling: Boolean = false
     }
 
     /**
@@ -961,7 +949,7 @@
         when (val state = this@JobSupport.state) {
             is ChildHandleNode -> yield(state.childJob)
             is Incomplete -> state.list?.let { list ->
-                list.forEach<ChildHandleNode> { yield(it.childJob) }
+                list.forEach { if (it is ChildHandleNode) yield(it.childJob) }
             }
         }
     }
@@ -977,7 +965,7 @@
          * If child is attached when the job is already being cancelled, such child will receive immediate notification on
          * cancellation, but parent *will* wait for that child before completion and will handle its exception.
          */
-        return invokeOnCompletion(onCancelling = true, handler = ChildHandleNode(child)) as ChildHandle
+        return invokeOnCompletion(handler = ChildHandleNode(child)) as ChildHandle
     }
 
     /**
@@ -1164,6 +1152,7 @@
         override fun invoke(cause: Throwable?) {
             parent.continueCompleting(state, child, proposedUpdate)
         }
+        override val onCancelling: Boolean get() = false
     }
 
     private class AwaitContinuation<T>(
@@ -1181,7 +1170,7 @@
             return parent.getCancellationException()
         }
 
-        protected override fun nameString(): String =
+        override fun nameString(): String =
             "AwaitContinuation"
     }
 
@@ -1280,6 +1269,7 @@
             val result = if (state is CompletedExceptionally) state else state.unboxState()
             select.trySelect(this@JobSupport, result)
         }
+        override val onCancelling: Boolean get() = false
     }
 }
 
@@ -1347,15 +1337,44 @@
     val list: NodeList? // is null only for Empty and JobNode incomplete state objects
 }
 
-internal abstract class JobNode : LockFreeLinkedListNode(), InternalCompletionHandler, DisposableHandle, Incomplete {
+internal abstract class JobNode : LockFreeLinkedListNode(), DisposableHandle, Incomplete {
     /**
-     * Initialized by [JobSupport.makeNode].
+     * Initialized by [JobSupport.invokeOnCompletionInternal].
      */
     lateinit var job: JobSupport
     override val isActive: Boolean get() = true
     override val list: NodeList? get() = null
     override fun dispose() = job.removeNode(this)
     override fun toString() = "$classSimpleName@$hexAddress[job@${job.hexAddress}]"
+    /**
+     * Signals completion.
+     *
+     * This function:
+     * - Does not throw any exceptions.
+     *   For [Job] instances that are coroutines, exceptions thrown by this function will be caught, wrapped into
+     *   [CompletionHandlerException], and passed to [handleCoroutineException], but for those that are not coroutines,
+     *   they will just be rethrown, potentially crashing unrelated code.
+     * - Is fast, non-blocking, and thread-safe.
+     * - Can be invoked concurrently with the surrounding code.
+     * - Can be invoked from any context.
+     *
+     * The meaning of `cause` that is passed to the handler is:
+     * - It is `null` if the job has completed normally.
+     * - It is an instance of [CancellationException] if the job was cancelled _normally_.
+     *   **It should not be treated as an error**. In particular, it should not be reported to error logs.
+     * - Otherwise, the job had _failed_.
+     *
+     * [CompletionHandler] is the user-visible interface for supplying custom implementations of [invoke]
+     * (see [InvokeOnCompletion] and [InvokeOnCancelling]).
+     */
+    abstract fun invoke(cause: Throwable?)
+
+    /**
+     * If `false`, [invoke] will be called once the job is cancelled or is complete.
+     * If `true`, [invoke] is invoked as soon as the job becomes _cancelling_ instead, and if that doesn't happen,
+     * it will be called once the job is cancelled or is complete.
+     */
+    abstract val onCancelling: Boolean
 }
 
 internal class NodeList : LockFreeLinkedListHead(), Incomplete {
@@ -1367,9 +1386,11 @@
         append(state)
         append("}[")
         var first = true
-        this@NodeList.forEach<JobNode> { node ->
-            if (first) first = false else append(", ")
-            append(node)
+        this@NodeList.forEach { node ->
+            if (node is JobNode) {
+                if (first) first = false else append(", ")
+                append(node)
+            }
         }
         append("]")
     }
@@ -1378,7 +1399,7 @@
         if (DEBUG) getString("Active") else super.toString()
 }
 
-internal class InactiveNodeList(
+private class InactiveNodeList(
     override val list: NodeList
 ) : Incomplete {
     override val isActive: Boolean get() = false
@@ -1386,15 +1407,17 @@
 }
 
 private class InvokeOnCompletion(
-    private val handler: InternalCompletionHandler
+    private val handler: CompletionHandler
 ) : JobNode()  {
     override fun invoke(cause: Throwable?) = handler.invoke(cause)
+    override val onCancelling = false
 }
 
 private class ResumeOnCompletion(
     private val continuation: Continuation<Unit>
 ) : JobNode() {
     override fun invoke(cause: Throwable?) = continuation.resume(Unit)
+    override val onCancelling = false
 }
 
 private class ResumeAwaitOnCompletion<T>(
@@ -1412,36 +1435,27 @@
             continuation.resume(state.unboxState() as T)
         }
     }
-}
-
-internal class DisposeOnCompletion(
-    private val handle: DisposableHandle
-) : JobNode() {
-    override fun invoke(cause: Throwable?) = handle.dispose()
+    override val onCancelling = false
 }
 
 // -------- invokeOnCancellation nodes
 
-/**
- * Marker for node that shall be invoked on in _cancelling_ state.
- * **Note: may be invoked multiple times.**
- */
-internal abstract class JobCancellingNode : JobNode()
-
 private class InvokeOnCancelling(
-    private val handler: InternalCompletionHandler
-) : JobCancellingNode()  {
+    private val handler: CompletionHandler
+) : JobNode()  {
     // delegate handler shall be invoked at most once, so here is an additional flag
-    private val _invoked = atomic(0) // todo: replace with atomic boolean after migration to recent atomicFu
+    private val _invoked = atomic(false)
     override fun invoke(cause: Throwable?) {
-        if (_invoked.compareAndSet(0, 1)) handler.invoke(cause)
+        if (_invoked.compareAndSet(expect = false, update = true)) handler.invoke(cause)
     }
+    override val onCancelling = true
 }
 
 private class ChildHandleNode(
     @JvmField val childJob: ChildJob
-) : JobCancellingNode(), ChildHandle {
+) : JobNode(), ChildHandle {
     override val parent: Job get() = job
     override fun invoke(cause: Throwable?) = childJob.parentCancelled(job)
     override fun childCancelled(cause: Throwable): Boolean = job.childCancelled(cause)
+    override val onCancelling: Boolean = true
 }
diff --git a/kotlinx-coroutines-core/common/src/internal/Atomic.kt b/kotlinx-coroutines-core/common/src/internal/Atomic.kt
index 60e06ec..eddddc7 100644
--- a/kotlinx-coroutines-core/common/src/internal/Atomic.kt
+++ b/kotlinx-coroutines-core/common/src/internal/Atomic.kt
@@ -19,12 +19,6 @@
      */
     abstract fun perform(affected: Any?): Any?
 
-    /**
-     * Returns reference to atomic operation that this descriptor is a part of or `null`
-     * if not a part of any [AtomicOp].
-     */
-    abstract val atomicOp: AtomicOp<*>?
-
     override fun toString(): String = "$classSimpleName@$hexAddress" // debug
 }
 
@@ -46,8 +40,6 @@
 public abstract class AtomicOp<in T> : OpDescriptor() {
     private val _consensus = atomic<Any?>(NO_DECISION)
 
-    override val atomicOp: AtomicOp<*> get() = this
-
     private fun decide(decision: Any?): Any? {
         assert { decision !== NO_DECISION }
         val current = _consensus.value
diff --git a/kotlinx-coroutines-core/common/src/internal/LockFreeLinkedList.common.kt b/kotlinx-coroutines-core/common/src/internal/LockFreeLinkedList.common.kt
index 456a3b7..7f9ee66 100644
--- a/kotlinx-coroutines-core/common/src/internal/LockFreeLinkedList.common.kt
+++ b/kotlinx-coroutines-core/common/src/internal/LockFreeLinkedList.common.kt
@@ -10,16 +10,16 @@
     public val isRemoved: Boolean
     public val nextNode: LockFreeLinkedListNode
     public val prevNode: LockFreeLinkedListNode
-    public fun addLast(node: LockFreeLinkedListNode)
     public fun addOneIfEmpty(node: LockFreeLinkedListNode): Boolean
     public inline fun addLastIf(node: LockFreeLinkedListNode, crossinline condition: () -> Boolean): Boolean
     public open fun remove(): Boolean
 
 }
 
+internal fun LockFreeLinkedListNode.addLast(node: LockFreeLinkedListNode) = addLastIf(node) { true }
+
 /** @suppress **This is unstable API and it is subject to change.** */
 public expect open class LockFreeLinkedListHead() : LockFreeLinkedListNode {
-    public val isEmpty: Boolean
-    public inline fun <reified T : LockFreeLinkedListNode> forEach(block: (T) -> Unit)
+    public inline fun forEach(block: (LockFreeLinkedListNode) -> Unit)
     public final override fun remove(): Nothing
 }
diff --git a/kotlinx-coroutines-core/concurrent/src/internal/LockFreeLinkedList.kt b/kotlinx-coroutines-core/concurrent/src/internal/LockFreeLinkedList.kt
index 970c01f..b2d069e 100644
--- a/kotlinx-coroutines-core/concurrent/src/internal/LockFreeLinkedList.kt
+++ b/kotlinx-coroutines-core/concurrent/src/internal/LockFreeLinkedList.kt
@@ -91,7 +91,7 @@
     // prev.next correction, which does not provide linearizable backwards iteration, but can be used to
     // resume forward iteration when current node was removed.
     public actual val prevNode: Node
-        get() = correctPrev(null) ?: findPrevNonRemoved(_prev.value)
+        get() = correctPrev() ?: findPrevNonRemoved(_prev.value)
 
     private tailrec fun findPrevNonRemoved(current: Node): Node {
         if (!current.isRemoved) return current
@@ -117,15 +117,6 @@
     // ------ addLastXXX ------
 
     /**
-     * Adds last item to this list.
-     */
-    public actual fun addLast(node: Node) {
-        while (true) { // lock-free loop on prev.next
-            if (prevNode.addNext(node, this)) return
-        }
-    }
-
-    /**
      * Adds last item to this list atomically if the [condition] is true.
      */
     public actual inline fun addLastIf(node: Node, crossinline condition: () -> Boolean): Boolean {
@@ -207,7 +198,7 @@
             val removed = (next as Node).removed()
             if (_next.compareAndSet(next, removed)) {
                 // was removed successfully (linearized remove) -- fixup the list
-                next.correctPrev(null)
+                next.correctPrev()
                 return null
             }
         }
@@ -247,14 +238,12 @@
             if (next._prev.compareAndSet(nextPrev, this)) {
                 // This newly added node could have been removed, and the above CAS would have added it physically again.
                 // Let us double-check for this situation and correct if needed
-                if (isRemoved) next.correctPrev(null)
+                if (isRemoved) next.correctPrev()
                 return
             }
         }
     }
 
-    protected open fun nextIfRemoved(): Node? = (next as? Removed)?.ref
-
     /**
      * Returns the corrected value of the previous node while also correcting the `prev` pointer
      * (so that `this.prev.next === this`) and helps complete node removals to the left ot this node.
@@ -265,7 +254,7 @@
      *   remover of this node will ultimately call [correctPrev] on the next node and that will fix all
      *   the links from this node, too.
      */
-    private tailrec fun correctPrev(op: OpDescriptor?): Node? {
+    private tailrec fun correctPrev(): Node? {
         val oldPrev = _prev.value
         var prev: Node = oldPrev
         var last: Node? = null // will be set so that last.next === prev
@@ -278,22 +267,21 @@
                     // otherwise need to update prev
                     if (!this._prev.compareAndSet(oldPrev, prev)) {
                         // Note: retry from scratch on failure to update prev
-                        return correctPrev(op)
+                        return correctPrev()
                     }
                     return prev // return the correct prev
                 }
                 // slow path when we need to help remove operations
                 this.isRemoved -> return null // nothing to do, this node was removed, bail out asap to save time
-                prevNext === op -> return prev // part of the same op -- don't recurse, didn't correct prev
                 prevNext is OpDescriptor -> { // help & retry
                     prevNext.perform(prev)
-                    return correctPrev(op) // retry from scratch
+                    return correctPrev() // retry from scratch
                 }
                 prevNext is Removed -> {
                     if (last !== null) {
                         // newly added (prev) node is already removed, correct last.next around it
                         if (!last._next.compareAndSet(prev, prevNext.ref)) {
-                            return correctPrev(op) // retry from scratch on failure to update next
+                            return correctPrev() // retry from scratch on failure to update next
                         }
                         prev = last
                         last = null
@@ -327,15 +315,13 @@
  * @suppress **This is unstable API and it is subject to change.**
  */
 public actual open class LockFreeLinkedListHead : LockFreeLinkedListNode() {
-    public actual val isEmpty: Boolean get() = next === this
-
     /**
      * Iterates over all elements in this list of a specified type.
      */
-    public actual inline fun <reified T : Node> forEach(block: (T) -> Unit) {
+    public actual inline fun forEach(block: (Node) -> Unit) {
         var cur: Node = next as Node
         while (cur != this) {
-            if (cur is T) block(cur)
+            block(cur)
             cur = cur.nextNode
         }
     }
@@ -345,6 +331,4 @@
 
     // optimization: because head is never removed, we don't have to read _next.value to check these:
     override val isRemoved: Boolean get() = false
-
-    override fun nextIfRemoved(): Node? = null
 }
diff --git a/kotlinx-coroutines-core/jsAndWasmShared/src/internal/LinkedList.kt b/kotlinx-coroutines-core/jsAndWasmShared/src/internal/LinkedList.kt
index 8d68f8b..6e81c79 100644
--- a/kotlinx-coroutines-core/jsAndWasmShared/src/internal/LinkedList.kt
+++ b/kotlinx-coroutines-core/jsAndWasmShared/src/internal/LinkedList.kt
@@ -2,27 +2,20 @@
 
 package kotlinx.coroutines.internal
 
-import kotlinx.coroutines.*
-
-private typealias Node = LinkedListNode
+private typealias Node = LockFreeLinkedListNode
 
 /** @suppress **This is unstable API and it is subject to change.** */
-public actual typealias LockFreeLinkedListNode = LinkedListNode
-
-/** @suppress **This is unstable API and it is subject to change.** */
-public actual typealias LockFreeLinkedListHead = LinkedListHead
-
-/** @suppress **This is unstable API and it is subject to change.** */
-public open class LinkedListNode : DisposableHandle {
+public actual open class LockFreeLinkedListNode {
     @PublishedApi internal var _next = this
     @PublishedApi internal var _prev = this
     @PublishedApi internal var _removed: Boolean = false
 
-    public inline val nextNode get() = _next
-    public inline val prevNode get() = _prev
-    public inline val isRemoved get() = _removed
+    public actual inline val nextNode get() = _next
+    inline actual val prevNode get() = _prev
+    inline actual val isRemoved get() = _removed
 
-    public fun addLast(node: Node) {
+    @PublishedApi
+    internal fun addLast(node: Node) {
         val prev = this._prev
         node._next = this
         node._prev = prev
@@ -36,16 +29,11 @@
      * I.g. `LockFreeLinkedListHead` throws, while `SendElementWithUndeliveredHandler`
      * invokes handler on remove
      */
-    public open fun remove(): Boolean {
+    public actual open fun remove(): Boolean {
         return removeImpl()
     }
 
-    override fun dispose() {
-        remove()
-    }
-
-    @PublishedApi
-    internal fun removeImpl(): Boolean {
+    private fun removeImpl(): Boolean {
         if (_removed) return false
         val prev = this._prev
         val next = this._next
@@ -55,60 +43,32 @@
         return true
     }
 
-    public fun addOneIfEmpty(node: Node): Boolean {
+    public actual fun addOneIfEmpty(node: Node): Boolean {
         if (_next !== this) return false
         addLast(node)
         return true
     }
 
-    public inline fun addLastIf(node: Node, crossinline condition: () -> Boolean): Boolean {
+    public actual inline fun addLastIf(node: Node, crossinline condition: () -> Boolean): Boolean {
         if (!condition()) return false
         addLast(node)
         return true
     }
-
-    public inline fun addLastIfPrev(node: Node, predicate: (Node) -> Boolean): Boolean {
-        if (!predicate(_prev)) return false
-        addLast(node)
-        return true
-    }
-
-    public inline fun addLastIfPrevAndIf(
-        node: Node,
-        predicate: (Node) -> Boolean, // prev node predicate
-        crossinline condition: () -> Boolean // atomically checked condition
-    ): Boolean {
-        if (!predicate(_prev)) return false
-        if (!condition()) return false
-        addLast(node)
-        return true
-    }
-
-    public fun helpRemove() {} // No concurrency on JS -> no removal
-
-    public fun removeFirstOrNull(): Node? {
-        val next = _next
-        if (next === this) return null
-        check(next.removeImpl()) { "Should remove" }
-        return next
-    }
 }
 
 /** @suppress **This is unstable API and it is subject to change.** */
-public open class LinkedListHead : LinkedListNode() {
-    public val isEmpty get() = _next === this
-
+public actual open class LockFreeLinkedListHead : Node() {
     /**
      * Iterates over all elements in this list of a specified type.
      */
-    public inline fun <reified T : Node> forEach(block: (T) -> Unit) {
+    public actual inline fun forEach(block: (Node) -> Unit) {
         var cur: Node = _next
         while (cur != this) {
-            if (cur is T) block(cur)
+            block(cur)
             cur = cur._next
         }
     }
 
     // just a defensive programming -- makes sure that list head sentinel is never removed
-    public final override fun remove(): Nothing = throw UnsupportedOperationException()
+    public actual final override fun remove(): Nothing = throw UnsupportedOperationException()
 }
diff --git a/kotlinx-coroutines-core/jsAndWasmShared/test/internal/LinkedListTest.kt b/kotlinx-coroutines-core/jsAndWasmShared/test/internal/LinkedListTest.kt
index 04b7006..d88ddf1 100644
--- a/kotlinx-coroutines-core/jsAndWasmShared/test/internal/LinkedListTest.kt
+++ b/kotlinx-coroutines-core/jsAndWasmShared/test/internal/LinkedListTest.kt
@@ -6,11 +6,11 @@
 import kotlin.test.assertTrue
 
 class LinkedListTest {
-    data class IntNode(val i: Int) : LinkedListNode()
+    data class IntNode(val i: Int) : LockFreeLinkedListNode()
 
     @Test
     fun testSimpleAddLastRemove() {
-        val list = LinkedListHead()
+        val list = LockFreeLinkedListHead()
         assertContents(list)
         val n1 = IntNode(1).apply { list.addLast(this) }
         assertContents(list, 1)
@@ -31,13 +31,12 @@
         assertContents(list)
     }
 
-    private fun assertContents(list: LinkedListHead, vararg expected: Int) {
+    private fun assertContents(list: LockFreeLinkedListHead, vararg expected: Int) {
         val n = expected.size
         val actual = IntArray(n)
         var index = 0
-        list.forEach<IntNode> { actual[index++] = it.i }
+        list.forEach { if (it is IntNode) actual[index++] = it.i }
         assertEquals(n, index)
         for (i in 0 until n) assertEquals(expected[i], actual[i], "item i")
-        assertEquals(expected.isEmpty(), list.isEmpty)
     }
 }
diff --git a/kotlinx-coroutines-core/jvm/src/Future.kt b/kotlinx-coroutines-core/jvm/src/Future.kt
index 2e217cc..85d3cea 100644
--- a/kotlinx-coroutines-core/jvm/src/Future.kt
+++ b/kotlinx-coroutines-core/jvm/src/Future.kt
@@ -36,6 +36,8 @@
         // interruption flag and it will cause spurious failures elsewhere
         if (cause != null) future.cancel(false)
     }
+
+    override val onCancelling = false
 }
 
 private class CancelFutureOnCancel(private val future: Future<*>) : CancelHandler {
diff --git a/kotlinx-coroutines-core/jvm/src/Interruptible.kt b/kotlinx-coroutines-core/jvm/src/Interruptible.kt
index b25a289..3013d36 100644
--- a/kotlinx-coroutines-core/jvm/src/Interruptible.kt
+++ b/kotlinx-coroutines-core/jvm/src/Interruptible.kt
@@ -42,8 +42,8 @@
 
 private fun <T> runInterruptibleInExpectedContext(coroutineContext: CoroutineContext, block: () -> T): T {
     try {
-        val threadState = ThreadState(coroutineContext.job)
-        threadState.setup()
+        val threadState = ThreadState()
+        threadState.setup(coroutineContext.job)
         try {
             return block()
         } finally {
@@ -59,7 +59,7 @@
 private const val INTERRUPTING = 2
 private const val INTERRUPTED = 3
 
-private class ThreadState(private val job: Job) : InternalCompletionHandler {
+private class ThreadState : JobNode() {
     /*
        === States ===
 
@@ -95,8 +95,8 @@
     // Registered cancellation handler
     private var cancelHandle: DisposableHandle? = null
 
-    fun setup() {
-        cancelHandle = job.invokeOnCompletion(onCancelling = true, invokeImmediately = true, handler = this)
+    fun setup(job: Job) {
+        cancelHandle = job.invokeOnCompletion(handler = this)
         // Either we successfully stored it or it was immediately cancelled
         _state.loop { state ->
             when (state) {
@@ -154,5 +154,7 @@
         }
     }
 
+    override val onCancelling = true
+
     private fun invalidState(state: Int): Nothing = error("Illegal state $state")
 }
diff --git a/kotlinx-coroutines-core/jvm/test/internal/LockFreeLinkedListLongStressTest.kt b/kotlinx-coroutines-core/jvm/test/internal/LockFreeLinkedListLongStressTest.kt
index 577e73d..21adeb4 100644
--- a/kotlinx-coroutines-core/jvm/test/internal/LockFreeLinkedListLongStressTest.kt
+++ b/kotlinx-coroutines-core/jvm/test/internal/LockFreeLinkedListLongStressTest.kt
@@ -41,8 +41,8 @@
                 val rnd = Random()
                 do {
                     val lastTurn = workingAdders.get() == 0
-                    list.forEach<IntNode> { node ->
-                        if (shallRemove(node.i) && (lastTurn || rnd.nextDouble() < removeProbability))
+                    list.forEach { node ->
+                        if (node is IntNode && shallRemove(node.i) && (lastTurn || rnd.nextDouble() < removeProbability))
                             node.remove()
                     }
                 } while (!lastTurn)
@@ -62,8 +62,8 @@
                 if (!shallRemove(i))
                     yield(i)
         }
-        list.forEach<IntNode> { node ->
-            require(node.i == expected.next())
+        list.forEach { node ->
+            require(node !is IntNode || node.i == expected.next())
         }
         require(!expected.hasNext())
     }
diff --git a/reactive/kotlinx-coroutines-reactor/src/Scheduler.kt b/reactive/kotlinx-coroutines-reactor/src/Scheduler.kt
index d780f51..5371ff3 100644
--- a/reactive/kotlinx-coroutines-reactor/src/Scheduler.kt
+++ b/reactive/kotlinx-coroutines-reactor/src/Scheduler.kt
@@ -47,6 +47,4 @@
 }
 
 private fun Disposable.asDisposableHandle(): DisposableHandle =
-    object : DisposableHandle {
-        override fun dispose() = this@asDisposableHandle.dispose()
-    }
+    DisposableHandle { this@asDisposableHandle.dispose() }