Fix exception aggregation to ensure atomic handling of exceptions

* Removed legacy onFinishing handler support from JobSupport.
  - It is no longer needed, because handleJobException covers #208
* Fixed bugs that were masked by cancelling parent from onFinishing.
* Consistent "Finishing" state was introduced in internal machinery:
  - Job enters finishing state when it is failing or it is completing
    and has children
  - Finishing state cleanly aggregates all failures, tracks cancellation
    and completion status
* Job.isFailed is introduced as a consistent way to query the "failing"
  state of the job that was previously only implicitly available via
  invokeOnCompletion handler (cause != null means a failed job) and
  the documentation for both Job & Deferred is updated to reflect that.
* Source-incompatible change: Job.invokeOnCompletion boolean parameter is
  change to onFailing. Such handlers are now invoked as soon as the job
  starts failing and the root cause exception of the failure is consistently
  passed to all the handlers.
* The following internal methods were introduced to facilitate this:
  - Job.failParent(exception) is used by child to signal failure to parent
  - Job.cancelChild(parentJob) is used by parent to cancel child.
* Child never aggregates exception received from it parent, but uses
  it as it root cause if there is no other exception.
* JobSupport.handleJobException() logic for launch/actor is split into:
  - failParent - can be invoked multiple times on race;
  - handleJobException which is invoked exactly once.
* Exception materiazization is much lazier now, which should
  significantly improve performance when cancelling large hierarchies.
* Other minor perf improvements in JobSupport code.

Fixes #585
diff --git a/binary-compatibility-validator/reference-public-api/kotlinx-coroutines-core.txt b/binary-compatibility-validator/reference-public-api/kotlinx-coroutines-core.txt
index f95bce0..9596fe3 100644
--- a/binary-compatibility-validator/reference-public-api/kotlinx-coroutines-core.txt
+++ b/binary-compatibility-validator/reference-public-api/kotlinx-coroutines-core.txt
@@ -1,4 +1,5 @@
 public abstract class kotlinx/coroutines/experimental/AbstractCoroutine : kotlin/coroutines/experimental/Continuation, kotlinx/coroutines/experimental/CoroutineScope, kotlinx/coroutines/experimental/Job {
+	protected final field parentContext Lkotlin/coroutines/experimental/CoroutineContext;
 	public fun <init> (Lkotlin/coroutines/experimental/CoroutineContext;Z)V
 	public synthetic fun <init> (Lkotlin/coroutines/experimental/CoroutineContext;ZILkotlin/jvm/internal/DefaultConstructorMarker;)V
 	public final fun getContext ()Lkotlin/coroutines/experimental/CoroutineContext;
@@ -369,7 +370,9 @@
 	public static final field Key Lkotlinx/coroutines/experimental/Job$Key;
 	public abstract fun attachChild (Lkotlinx/coroutines/experimental/Job;)Lkotlinx/coroutines/experimental/DisposableHandle;
 	public abstract fun cancel (Ljava/lang/Throwable;)Z
+	public abstract fun cancelChild (Lkotlinx/coroutines/experimental/Job;)V
 	public abstract synthetic fun cancelChildren (Ljava/lang/Throwable;)V
+	public abstract fun childFailed (Ljava/lang/Throwable;)Z
 	public abstract fun getCancellationException ()Ljava/util/concurrent/CancellationException;
 	public abstract fun getChildren ()Lkotlin/sequences/Sequence;
 	public abstract fun getCompletionException ()Ljava/lang/Throwable;
@@ -381,6 +384,7 @@
 	public abstract fun isActive ()Z
 	public abstract fun isCancelled ()Z
 	public abstract fun isCompleted ()Z
+	public abstract fun isFailed ()Z
 	public abstract fun join (Lkotlin/coroutines/experimental/Continuation;)Ljava/lang/Object;
 	public abstract fun plus (Lkotlinx/coroutines/experimental/Job;)Lkotlinx/coroutines/experimental/Job;
 	public abstract fun start ()Z
@@ -440,7 +444,9 @@
 	public static final field INSTANCE Lkotlinx/coroutines/experimental/NonCancellable;
 	public fun attachChild (Lkotlinx/coroutines/experimental/Job;)Lkotlinx/coroutines/experimental/DisposableHandle;
 	public fun cancel (Ljava/lang/Throwable;)Z
+	public fun cancelChild (Lkotlinx/coroutines/experimental/Job;)V
 	public synthetic fun cancelChildren (Ljava/lang/Throwable;)V
+	public fun childFailed (Ljava/lang/Throwable;)Z
 	public fun getCancellationException ()Ljava/util/concurrent/CancellationException;
 	public fun getChildren ()Lkotlin/sequences/Sequence;
 	public fun getCompletionException ()Ljava/lang/Throwable;
@@ -452,6 +458,7 @@
 	public fun isActive ()Z
 	public fun isCancelled ()Z
 	public fun isCompleted ()Z
+	public fun isFailed ()Z
 	public fun join (Lkotlin/coroutines/experimental/Continuation;)Ljava/lang/Object;
 	public fun plus (Lkotlinx/coroutines/experimental/Job;)Lkotlinx/coroutines/experimental/Job;
 	public fun start ()Z
diff --git a/common/kotlinx-coroutines-core-common/src/AbstractContinuation.kt b/common/kotlinx-coroutines-core-common/src/AbstractContinuation.kt
index 95d7b7b..48ece0f 100644
--- a/common/kotlinx-coroutines-core-common/src/AbstractContinuation.kt
+++ b/common/kotlinx-coroutines-core-common/src/AbstractContinuation.kt
@@ -89,7 +89,7 @@
             return
         }
         parent.start() // make sure the parent is started
-        val handle = parent.invokeOnCompletion(onCancelling = true,
+        val handle = parent.invokeOnCompletion(onFailing = true,
             handler = ChildContinuation(parent, this).asHandler)
 
         parentHandle = handle
diff --git a/common/kotlinx-coroutines-core-common/src/AbstractCoroutine.kt b/common/kotlinx-coroutines-core-common/src/AbstractCoroutine.kt
index b6f840b..d83cf92 100644
--- a/common/kotlinx-coroutines-core-common/src/AbstractCoroutine.kt
+++ b/common/kotlinx-coroutines-core-common/src/AbstractCoroutine.kt
@@ -5,6 +5,7 @@
 package kotlinx.coroutines.experimental
 
 import kotlinx.coroutines.experimental.CoroutineStart.*
+import kotlinx.coroutines.experimental.internal.*
 import kotlinx.coroutines.experimental.intrinsics.*
 import kotlin.coroutines.experimental.*
 
@@ -30,7 +31,11 @@
  */
 @Suppress("EXPOSED_SUPER_CLASS")
 public abstract class AbstractCoroutine<in T>(
-    private val parentContext: CoroutineContext,
+    /**
+     * Context of the parent coroutine.
+     */
+    @JvmField
+    protected val parentContext: CoroutineContext,
     active: Boolean = true
 ) : JobSupport(active), Job, Continuation<T>, CoroutineScope {
     @Suppress("LeakingThis")
@@ -63,19 +68,25 @@
     }
 
     /**
-     * This function is invoked once when this coroutine is cancelled or is completed,
-     * similarly to [invokeOnCompletion] with `onCancelling` set to `true`.
+     * @suppress **Deprecated**: Override [onFailing].
+     */
+    @Deprecated("Override onFailing")
+    protected open fun onCancellation(cause: Throwable?) {}
+
+    /**
+     * This function is invoked once when this coroutine is failing or is completed,
+     * similarly to [invokeOnCompletion] with `onFailing` set to `true`.
      *
      * The meaning of [cause] parameter:
      * * Cause is `null` when job has completed normally.
      * * Cause is an instance of [CancellationException] when 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_.
+     *
+     * @suppress **Deprecated**: Override [onFailing].
      */
-    protected open fun onCancellation(cause: Throwable?) {}
-
-    internal override fun onCancellationInternal(exceptionally: CompletedExceptionally?) {
-        onCancellation(exceptionally?.cause)
+    override fun onFailing(cause: Throwable?) {
+        onCancellation(cause)
     }
 
     /**
@@ -89,7 +100,7 @@
     protected open fun onCompletedExceptionally(exception: Throwable) {}
 
     @Suppress("UNCHECKED_CAST")
-    internal override fun onCompletionInternal(state: Any?, mode: Int) {
+    internal override fun onCompletionInternal(state: Any?, mode: Int, suppressed: Boolean) {
         if (state is CompletedExceptionally)
             onCompletedExceptionally(state.cause)
         else
@@ -112,6 +123,13 @@
         makeCompletingOnce(CompletedExceptionally(exception), defaultResumeMode)
     }
 
+    // todo: make it for all kinds of coroutines, now only launch & actor override and handleExceptionViaJob
+    internal fun failParentImpl(exception: Throwable): Boolean {
+        if (exception is CancellationException) return true
+        val parentJob = parentContext[Job]
+        return parentJob !== null && parentJob.childFailed(exception)
+    }
+
     internal final override fun handleOnCompletionException(exception: Throwable) {
         handleCoroutineException(parentContext, exception, this)
     }
diff --git a/common/kotlinx-coroutines-core-common/src/Builders.common.kt b/common/kotlinx-coroutines-core-common/src/Builders.common.kt
index 4bfba1d..24f4c1a 100644
--- a/common/kotlinx-coroutines-core-common/src/Builders.common.kt
+++ b/common/kotlinx-coroutines-core-common/src/Builders.common.kt
@@ -207,20 +207,11 @@
 // --------------- implementation ---------------
 
 private open class StandaloneCoroutine(
-    private val parentContext: CoroutineContext,
+    parentContext: CoroutineContext,
     active: Boolean
 ) : AbstractCoroutine<Unit>(parentContext, active) {
-    override fun hasOnFinishingHandler(update: Any?) = update is CompletedExceptionally
-
-    override fun handleJobException(exception: Throwable) {
-        handleCoroutineException(parentContext, exception, this)
-    }
-
-    override fun onFinishingInternal(update: Any?) {
-        if (update is CompletedExceptionally && update.cause !is CancellationException) {
-            parentContext[Job]?.cancel(update.cause)
-        }
-    }
+    override fun failParent(exception: Throwable) = failParentImpl(exception)
+    override fun handleJobException(exception: Throwable) = handleExceptionViaHandler(parentContext, exception)
 }
 
 private class LazyStandaloneCoroutine(
diff --git a/common/kotlinx-coroutines-core-common/src/CompletableDeferred.kt b/common/kotlinx-coroutines-core-common/src/CompletableDeferred.kt
index d85b784..3949851 100644
--- a/common/kotlinx-coroutines-core-common/src/CompletableDeferred.kt
+++ b/common/kotlinx-coroutines-core-common/src/CompletableDeferred.kt
@@ -61,7 +61,7 @@
     parent: Job?
 ) : JobSupport(true), CompletableDeferred<T>, SelectClause1<T> {
     init { initParentJobInternal(parent) }
-    override val onCancelMode: Int get() = ON_CANCEL_MAKE_COMPLETING
+    override val onFailComplete get() = true
     override fun getCompleted(): T = getCompletedInternal() as T
     override suspend fun await(): T = awaitInternal() as T
     override val onAwait: SelectClause1<T> get() = this
diff --git a/common/kotlinx-coroutines-core-common/src/CompletedExceptionally.kt b/common/kotlinx-coroutines-core-common/src/CompletedExceptionally.kt
index 8d5ff94..44becc8 100644
--- a/common/kotlinx-coroutines-core-common/src/CompletedExceptionally.kt
+++ b/common/kotlinx-coroutines-core-common/src/CompletedExceptionally.kt
@@ -11,10 +11,8 @@
  * Class for an internal state of a job that had completed exceptionally, including cancellation.
  *
  * **Note: This class cannot be used outside of internal coroutines framework**.
- * **Note: cannot be internal until we get rid of MutableDelegateContinuation in IO**
+ * **Note: cannot be internal and renamed until we get rid of MutableDelegateContinuation in IO**
  *
- * @param cause the exceptional completion cause. It's either original exceptional cause
- *        or artificial JobCancellationException if no cause was provided
  * @suppress **This is unstable API and it is subject to change.**
  */
 open class CompletedExceptionally(
@@ -28,14 +26,9 @@
  *
  * **Note: This class cannot be used outside of internal coroutines framework**.
  *
- * @param job the job that was cancelled.
- * @param cause the exceptional completion cause. If `cause` is null, then a [JobCancellationException] is created.
  * @suppress **This is unstable API and it is subject to change.**
  */
-internal class Cancelled(
-    job: Job,
-    cause: Throwable?
-) : CompletedExceptionally(cause ?: JobCancellationException("Job was cancelled normally", null, job))
+internal class Cancelled(cause: Throwable) : CompletedExceptionally(cause)
 
 /**
  * A specific subclass of [CompletedExceptionally] for cancelled [AbstractContinuation].
diff --git a/common/kotlinx-coroutines-core-common/src/CompletionHandler.common.kt b/common/kotlinx-coroutines-core-common/src/CompletionHandler.common.kt
index c92634f..89f6a97 100644
--- a/common/kotlinx-coroutines-core-common/src/CompletionHandler.common.kt
+++ b/common/kotlinx-coroutines-core-common/src/CompletionHandler.common.kt
@@ -43,3 +43,5 @@
 // :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
 internal expect fun CompletionHandler.invokeIt(cause: Throwable?)
+
+internal inline fun <reified T> CompletionHandler.isHandlerOf(): Boolean = this is T
diff --git a/common/kotlinx-coroutines-core-common/src/CoroutineExceptionHandler.kt b/common/kotlinx-coroutines-core-common/src/CoroutineExceptionHandler.kt
index 82c4963..cba8439 100644
--- a/common/kotlinx-coroutines-core-common/src/CoroutineExceptionHandler.kt
+++ b/common/kotlinx-coroutines-core-common/src/CoroutineExceptionHandler.kt
@@ -20,22 +20,27 @@
  * If invocation returned `true`, method terminates: now [Job] is responsible for handling an exception.
  * Otherwise, If there is [CoroutineExceptionHandler] in the context, it is used.
  * Otherwise all instances of [CoroutineExceptionHandler] found via [ServiceLoader] and [Thread.uncaughtExceptionHandler] are invoked
+ *
+ * todo: Deprecate/hide this function.
  */
 @JvmOverloads // binary compatibility
 public fun handleCoroutineException(context: CoroutineContext, exception: Throwable, caller: Job? = null) {
-    // if exception handling fails, make sure the original exception is not lost
+    if (!handleExceptionViaJob(context, exception, caller)) {
+        handleExceptionViaHandler(context, exception)
+    }
+}
+
+private fun handleExceptionViaJob(context: CoroutineContext, exception: Throwable, caller: Job?): Boolean {
+    // Ignore CancellationException (they are normal ways to terminate a coroutine)
+    if (exception is CancellationException) return true
+    // If job is successfully cancelled, we're done
+    val job = context[Job]
+    return job !== null && job !== caller && job.cancel(exception)
+}
+
+internal fun handleExceptionViaHandler(context: CoroutineContext, exception: Throwable) {
     try {
-        // Ignore CancellationException (they are normal ways to terminate a coroutine)
-        if (exception is CancellationException) {
-            return
-        }
-        // If parent is successfully cancelled, we're done, it is now its responsibility to handle the exception
-        val parent = context[Job]
-        // E.g. actor registers itself in the context, in that case we should invoke handler
-        if (parent !== null && parent !== caller && parent.cancel(exception)) {
-            return
-        }
-        // If not, invoke exception handler from the context
+        // Invoke exception handler from the context if present
         context[CoroutineExceptionHandler]?.let {
             it.handleException(context, exception)
             return
diff --git a/common/kotlinx-coroutines-core-common/src/CoroutineScope.kt b/common/kotlinx-coroutines-core-common/src/CoroutineScope.kt
index f3f0b3f..c1f394f 100644
--- a/common/kotlinx-coroutines-core-common/src/CoroutineScope.kt
+++ b/common/kotlinx-coroutines-core-common/src/CoroutineScope.kt
@@ -175,7 +175,7 @@
  */
 public suspend fun <R> coroutineScope(block: suspend CoroutineScope.() -> R): R {
     // todo: optimize implementation to a single allocated object
-    val owner = ScopeOwnerCoroutine<R>(coroutineContext)
+    val owner = ScopeCoroutine<R>(coroutineContext)
     owner.start(CoroutineStart.UNDISPATCHED, owner, block)
     owner.join()
     if (owner.isCancelled) {
diff --git a/common/kotlinx-coroutines-core-common/src/Deferred.kt b/common/kotlinx-coroutines-core-common/src/Deferred.kt
index cfc2ec3..5c93061 100644
--- a/common/kotlinx-coroutines-core-common/src/Deferred.kt
+++ b/common/kotlinx-coroutines-core-common/src/Deferred.kt
@@ -9,69 +9,35 @@
 import kotlin.coroutines.experimental.*
 
 /**
- * Deferred value is a non-blocking cancellable future.
+ * Deferred value is a non-blocking cancellable future &mdash; it is a [Job] that has a result.
  *
  * It is created with [async][CoroutineScope.async] coroutine builder or via constructor of [CompletableDeferred] class.
  * It is in [active][isActive] state while the value is being computed.
  *
- * Deferred value has the following states:
- *
- * | **State**                               | [isActive] | [isCompleted] | [isCompletedExceptionally] | [isCancelled] |
- * | --------------------------------------- | ---------- | ------------- | -------------------------- | ------------- |
- * | _New_ (optional initial state)          | `false`    | `false`       | `false`                    | `false`       |
- * | _Active_ (default initial state)        | `true`     | `false`       | `false`                    | `false`       |
- * | _Completing_ (optional transient state) | `true`     | `false`       | `false`                    | `false`       |
- * | _Cancelling_ (optional transient state) | `false`    | `false`       | `false`                    | `true`        |
- * | _Cancelled_ (final state)               | `false`    | `true`        | `true`                     | `true`        |
- * | _Resolved_  (final state)               | `false`    | `true`        | `false`                    | `false`       |
- * | _Failed_    (final state)               | `false`    | `true`        | `true`                     | `false`       |
+ * Deferred value has the same state machine as the [Job] with additional convenience methods to retrieve
+ * successful or failed result of the computation that was carried out. The result of the deferred is
+ * available when it is [completed][isCompleted] and can be retrieved by [await] method, which throws
+ * exception if the deferred had failed.
+ * A _failed_ deferred is considered to be [completed exceptionally][isCompletedExceptionally].
+ * The corresponding exception can be retrieved via [getCompletionExceptionOrNull] from a completed instance of deferred.
  *
  * Usually, a deferred value is created in _active_ state (it is created and started).
  * However, [async][CoroutineScope.async] coroutine builder has an optional `start` parameter that creates a deferred value in _new_ state
  * when this parameter is set to [CoroutineStart.LAZY].
  * Such a deferred can be be made _active_ by invoking [start], [join], or [await].
  *
- * A deferred can be _cancelled_ at any time with [cancel] function that forces it to transition to
- * _cancelling_ state immediately. Deferred that is not backed by a coroutine (see [CompletableDeferred]) and does not have
- * [children] becomes _cancelled_ on [cancel] immediately.
- * Otherwise, deferred becomes _cancelled_  when it finishes executing its code and
- * when all its children [complete][isCompleted].
- *
- * ```
- *                                                     wait children
- *    +-----+       start      +--------+   complete  +-------------+ finish +-----------+
- *    | New | ---------------> | Active | ----------> | Completing  | ---+-> | Resolved  |
- *    +-----+                  +--------+             +-------------+    |   |(completed)|
- *       |                         |                        |            |   +-----------+
- *       | cancel                  | cancel                 | cancel     |
- *       V                         V                        |            |   +-----------+
- *  +-----------+   finish   +------------+                 |            +-> |  Failed   |
- *  | Cancelled | <--------- | Cancelling | <---------------+                |(completed)|
- *  |(completed)|            +------------+                                  +-----------+
- *  +-----------+
- * ```
- *
  * A deferred value is a [Job]. A job in the
  * [coroutineContext](https://kotlinlang.org/api/latest/jvm/stdlib/kotlin.coroutines.experimental/coroutine-context.html)
  * of [async][CoroutineScope.async] builder represents the coroutine itself.
- * A deferred value is active while the coroutine is working and cancellation aborts the coroutine when
- * the coroutine is suspended on a _cancellable_ suspension point by throwing [CancellationException]
- * or the cancellation cause inside the coroutine.
- *
- * A deferred value can have a _parent_ job. A deferred value with a parent is cancelled when its parent is
- * cancelled or completes. Parent waits for all its [children] to complete in _completing_ or
- * _cancelling_ state. _Completing_ state is purely internal. For an outside observer a _completing_
- * deferred is still active, while internally it is waiting for its children.
  *
  * All functions on this interface and on all interfaces derived from it are **thread-safe** and can
  * be safely invoked from concurrent coroutines without external synchronization.
  */
 public interface Deferred<out T> : Job {
     /**
-     * Returns `true` if computation of this deferred value has _completed exceptionally_ -- it had
-     * either _failed_ with exception during computation or was [cancelled][cancel].
-     *
-     * It implies that [isActive] is `false` and [isCompleted] is `true`.
+     * Returns `true` if computation of this deferred value has _completed exceptionally_.
+     * It is `true` when both [isCompleted] and [isFailed] are true.
+     * It implies that [isActive] is `false`.
      */
     public val isCompletedExceptionally: Boolean
 
diff --git a/common/kotlinx-coroutines-core-common/src/Job.kt b/common/kotlinx-coroutines-core-common/src/Job.kt
index 2540ada..fe8ffce 100644
--- a/common/kotlinx-coroutines-core-common/src/Job.kt
+++ b/common/kotlinx-coroutines-core-common/src/Job.kt
@@ -14,59 +14,65 @@
 // --------------- core job interfaces ---------------
 
 /**
- * A background job. Conceptually, a job is a cancellable thing with a simple life-cycle that
- * culminates in its completion. Jobs can be arranged into parent-child hierarchies where cancellation
- * or completion of parent immediately cancels all its [children].
+ * A background job. Conceptually, a job is a cancellable thing with a life-cycle that
+ * culminates in its completion. Jobs can be arranged into parent-child hierarchies where failure or cancellation
+ * of parent immediately cancels all its [children].
  *
  * The most basic instances of [Job] are created with [launch][CoroutineScope.launch] coroutine builder or with a
- * `Job()` factory function.  Other coroutine builders and primitives like
- * [Deferred] also implement [Job] interface.
+ * `Job()` factory function.
+ * Conceptually, an execution of the job does not produce a result value. Jobs are launched solely for their
+ * side-effects. See [Deferred] interface for a job that produces a result.
  *
  * A job has the following states:
  *
- * | **State**                               | [isActive] | [isCompleted] | [isCancelled] |
- * | --------------------------------------- | ---------- | ------------- | ------------- |
- * | _New_ (optional initial state)          | `false`    | `false`       | `false`       |
- * | _Active_ (default initial state)        | `true`     | `false`       | `false`       |
- * | _Completing_ (optional transient state) | `true`     | `false`       | `false`       |
- * | _Cancelling_ (optional transient state) | `false`    | `false`       | `true`        |
- * | _Cancelled_ (final state)               | `false`    | `true`        | `true`        |
- * | _Completed_ (final state)               | `false`    | `true`        | `false`       |
+ * | **State**                        | [isActive] | [isCompleted] | [isFailed] | [isCancelled] |
+ * | -------------------------------- | ---------- | ------------- | ---------- | ------------- |
+ * | _New_ (optional initial state)   | `false`    | `false`       | `false`    | `false`       |
+ * | _Active_ (default initial state) | `true`     | `false`       | `false`    | `false`       |
+ * | _Completing_ (transient state)   | `true`     | `false`       | `false`    | `false`       |
+ * | _Failing_ (transient state)      | `false`    | `false`       | `true`     | `false`       |
+ * | _Cancelling_ (transient state)   | `false`    | `false`       | `true`     | `true`        |
+ * | _Completed_ (final state)        | `false`    | `true`        | `false`    | `false`       |
+ * | _Failed (final state)            | `false`    | `true`        | `true`     | `false`       |
+ * | _Cancelled_ (final state)        | `false`    | `true`        | `true`     | `true`        |
  *
  * Usually, a job is created in _active_ state (it is created and started). However, coroutine builders
  * that provide an optional `start` parameter create a coroutine in _new_ state when this parameter is set to
  * [CoroutineStart.LAZY]. Such a job can be made _active_ by invoking [start] or [join].
  *
- * A job can be _cancelled_ at any time with [cancel] function that forces it to transition to
- * _cancelling_ state immediately. Job that is not backed by a coroutine (see `Job()` function) and does not have
- * [children] becomes _cancelled_ on [cancel] immediately.
- * Otherwise, job becomes _cancelled_  when it finishes executing its code and
- * when all its children [complete][isCompleted].
+ * A job is _active_ while the coroutine is working. Failure of the job makes it _failing_.
+ * A job can be forced to fail by cancelling it at any time with [cancel] function that forces it to transition to
+ * _cancelling_ state immediately. The job becomes _cancelled_  when it finishes executing it work.
  *
  * ```
- *                                                      wait children
- *    +-----+       start      +--------+   complete   +-------------+  finish  +-----------+
- *    | New | ---------------> | Active | -----------> | Completing  | -------> | Completed |
- *    +-----+                  +--------+              +-------------+          +-----------+
- *       |                         |                         |
- *       | cancel                  | cancel                  | cancel
- *       V                         V                         |
- *  +-----------+   finish   +------------+                  |
- *  | Cancelled | <--------- | Cancelling | <----------------+
- *  |(completed)|            +------------+
- *  +-----------+
+ *                                       wait children
+ * +-----+ start  +--------+ complete   +-------------+  finish  +-----------+
+ * | New | -----> | Active | ---------> | Completing  | -------> | Completed |
+ * +-----+        +--------+            +-------------+          +-----------+
+ *                  | failure detected     |  |
+ *                  |     +----------------+  |
+ *                  |     |                   |
+ *                  V     V                   |
+ *              +------------+                |          finish  +----------+
+ *              |  Failing   | -------------- | ---------------> | Failed   |
+ *              +------------+                |                  +----------+
+ *                  | cancel invoked          |
+ *                  |     +-------------------+
+ *                  |     |
+ *                  V     V
+ *              +------------+                           finish  +-----------+
+ *              | Cancelling | --------------------------------> | Cancelled |
+ *              +------------+                                   +-----------+
  * ```
  *
- * A job in the
+ * A `Job` instance in the
  * [coroutineContext](https://kotlinlang.org/api/latest/jvm/stdlib/kotlin.coroutines.experimental/coroutine-context.html)
  * represents the coroutine itself.
- * A job is active while the coroutine is working and job's cancellation aborts the coroutine when
- * the coroutine is suspended on a _cancellable_ suspension point by throwing [CancellationException].
  *
- * A job can have a _parent_ job. A job with a parent is cancelled when its parent is cancelled or completes exceptionally.
- * Parent job waits for all its children to complete in _completing_ or _cancelling_ state.
- * _Completing_ state is purely internal to the job. For an outside observer a _completing_ job is still active,
- * while internally it is waiting for its children.
+ * A job can have a _parent_ job. A job with a parent is cancelled when its parent is cancelled or is failing.
+ * Parent job waits in _completing_, _failing_, or _cancelling_ state for all its children to complete before finishing.
+ * Note, that _completing_ state is purely internal to the job. For an outside observer a _completing_ job is still
+ * active, while internally it is waiting for its children.
  *
  * All functions on this interface and on all interfaces derived from it are **thread-safe** and can
  * be safely invoked from concurrent coroutines without external synchronization.
@@ -97,22 +103,32 @@
     // ------------ state query ------------
 
     /**
-     * Returns `true` when this job is active -- it was already started and has not completed or cancelled yet.
+     * Returns `true` when this job is active -- it was already started and has not completed or failed yet.
      * The job that is waiting for its [children] to complete is still considered to be active if it
-     * was not cancelled.
+     * has not failed and was not cancelled.
      */
     public val isActive: Boolean
 
     /**
-     * Returns `true` when this job has completed for any reason. A job that was cancelled and has
-     * finished its execution is also considered complete. Job becomes complete only after
+     * Returns `true` when this job has completed for any reason. A job that has failed or cancelled
+     * and has finished its execution is also considered complete. Job becomes complete only after
      * all its [children] complete.
      */
     public val isCompleted: Boolean
 
     /**
-     * Returns `true` if this job was [cancelled][cancel]. In the general case, it does not imply that the
-     * job has already [completed][isCompleted] (it may still be cancelling whatever it was doing).
+     * Returns `true` if the failure was detected while running this job and it cannot complete normally.
+     * In the general case, it does not imply that the
+     * job has already [completed][isCompleted], because it may still be finishing whatever it was doing and
+     * waiting for its [children] to complete.
+     */
+    public val isFailed: Boolean
+
+    /**
+     * Returns `true` if this job was [cancelled][cancel]. It implies that [isFailed] returns `true`, too.
+     * In the general case, it does not imply that the
+     * job has already [completed][isCompleted], because it may still be finishing whatever it was doing and
+     * failing for its [children] to complete.
      */
     public val isCancelled: Boolean
 
@@ -127,8 +143,9 @@
      * returned. The [JobCancellationException.cause] of the resulting [JobCancellationException] references
      * the original cancellation cause that was passed to [cancel] function.
      *
-     * This function throws [IllegalStateException] when invoked on a job that has not
-     * [completed][isCompleted] nor [cancelled][isCancelled] yet.
+     * This function throws [IllegalStateException] when invoked on a job that is still active.
+     *
+     * @suppress **This is unstable API and it is subject to change.**
      */
     public fun getCancellationException(): CancellationException
 
@@ -163,6 +180,25 @@
     public fun cancel(cause: Throwable? = null): Boolean
 
     // ------------ parent-child ------------
+    
+    /**
+     * Child is reporting failure to the parent by invoking this method.
+     * This method is invoked by the child twice. The first time child report its root cause as soon as possible,
+     * so that all its siblings and the parent can start finishing their work asap on failure. The second time
+     * child invokes this method when it had aggregated and determined its final termination cause.
+     *
+     * @suppress **This is unstable API and it is subject to change.**
+     */
+    public fun childFailed(cause: Throwable): Boolean
+
+    /**
+     * Cancels child job. This method is invoked by [parentJob] to cancel this child job.
+     * Child finds the cancellation cause using [getCancellationException] of the [parentJob].
+     * This method does nothing is the child is already being cancelled.
+     *
+     * @suppress **This is unstable API and it is subject to change.**
+     */
+    public fun cancelChild(parentJob: Job)
 
     /**
      * Returns a sequence of this job's children.
@@ -201,9 +237,9 @@
      * lookup a [Job] instance in the parent context and use this function to attach themselves as a child.
      * They also store a reference to the resulting [DisposableHandle] and dispose a handle when they complete.
      *
-     * @suppress This is an internal API. This method is too error prone for public API.
+     * @suppress **This is unstable API and it is subject to change.**
+     *           This is an internal API. This method is too error prone for public API.
      */
-    @Deprecated(message = "Start child coroutine with 'parent' parameter", level = DeprecationLevel.WARNING)
     public fun attachChild(child: Job): DisposableHandle
 
     /**
@@ -285,10 +321,10 @@
     public fun invokeOnCompletion(handler: CompletionHandler): DisposableHandle
 
     /**
-     * Registers handler that is **synchronously** invoked once on cancellation or completion of this job.
-     * When job is already cancelling or complete, then the handler is immediately invoked
+     * Registers handler that is **synchronously** invoked once on failure or completion of this job.
+     * When job is already failing or complete, then the handler is immediately invoked
      * with a job's cancellation cause or `null` unless [invokeImmediately] is set to false.
-     * Otherwise, handler will be invoked once when this job is cancelled or complete.
+     * Otherwise, handler will be invoked once when this job is failing or is complete.
      *
      * The meaning of `cause` that is passed to the handler:
      * * Cause is `null` when job has completed normally.
@@ -296,12 +332,9 @@
      *   **It should not be treated as an error**. In particular, it should not be reported to error logs.
      * * Otherwise, the job had _failed_.
      *
-     * Invocation of this handler on a transition to a transient _cancelling_ state
-     * is controlled by [onCancelling] boolean parameter.
-     * The handler is invoked on invocation of [cancel] when
-     * job becomes _cancelling_ if [onCancelling] parameter is set to `true`. However,
-     * when this [Job] is not backed by a coroutine, like [CompletableDeferred] or [CancellableContinuation]
-     * (both of which do not posses a _cancelling_ state), then the value of [onCancelling] parameter is ignored.
+     * Invocation of this handler on a transition to a _failing_ state
+     * is controlled by [onFailing] boolean parameter.
+     * The handler is invoked when the job is failing when [onFailing] parameter is set to `true`.
      *
      * The resulting [DisposableHandle] can be used to [dispose][DisposableHandle.dispose] the
      * registration of this handler and release its memory if its invocation is no longer needed.
@@ -316,15 +349,17 @@
      * This function should not be used in general application code.
      * Implementations of `CompletionHandler` must be fast and _lock-free_.
      *
-     * @param onCancelling when `true`, then the [handler] is invoked as soon as this job transitions to _cancelling_ state;
+     * @param onFailing when `true`, then the [handler] is invoked as soon as this job transitions to _failing_ state;
      *        when `false` then the [handler] is invoked only when it transitions to _completed_ state.
-     * @param invokeImmediately when `true` and this job is already in the desired state (depending on [onCancelling]),
+     * @param invokeImmediately when `true` and this job is already in the desired state (depending on [onFailing]),
      *        then the [handler] is immediately and synchronously invoked and [NonDisposableHandle] is returned;
      *        when `false` then [NonDisposableHandle] is returned, but the [handler] is not invoked.
      * @param handler the handler.
+     * 
+     * @suppress **This is unstable API and it is subject to change.**
      */
     public fun invokeOnCompletion(
-        onCancelling: Boolean = false,
+        onFailing: Boolean = false,
         invokeImmediately: Boolean = true,
         handler: CompletionHandler): DisposableHandle
 
diff --git a/common/kotlinx-coroutines-core-common/src/JobSupport.kt b/common/kotlinx-coroutines-core-common/src/JobSupport.kt
index 5edeeef..a8ae208 100644
--- a/common/kotlinx-coroutines-core-common/src/JobSupport.kt
+++ b/common/kotlinx-coroutines-core-common/src/JobSupport.kt
@@ -26,42 +26,42 @@
     /*
        === Internal states ===
 
-       name       state class    public state  description
-       ------     ------------   ------------  -----------
-       EMPTY_N    EmptyNew     : New           no listeners
-       EMPTY_A    EmptyActive  : Active        no listeners
-       SINGLE     JobNode      : Active        a single listener
-       SINGLE+    JobNode      : Active        a single listener + NodeList added as its next
-       LIST_N     NodeList     : New           a list of listeners (promoted once, does not got back to EmptyNew)
-       LIST_A     NodeList     : Active        a list of listeners (promoted once, does not got back to JobNode/EmptyActive)
-       COMPLETING Finishing    : Completing    has a list of listeners (promoted once from LIST_*)
-       CANCELLING Finishing    : Cancelling    has a list of listeners (promoted once from LIST_*)
-       FINAL_C    Cancelled    : Cancelled     cancelled (final state)
-       FINAL_F    Failed       : Completed     failed for other reason (final state)
-       FINAL_R    <any>        : Completed     produced some result
+       name       state class              public state  description
+       ------     ------------             ------------  -----------
+       EMPTY_N    EmptyNew               : New           no listeners
+       EMPTY_A    EmptyActive            : Active        no listeners
+       SINGLE     JobNode                : Active        a single listener
+       SINGLE+    JobNode                : Active        a single listener + NodeList added as its next
+       LIST_N     InactiveNodeList       : New           a list of listeners (promoted once, does not got back to EmptyNew)
+       LIST_A     NodeList               : Active        a list of listeners (promoted once, does not got back to JobNode/EmptyActive)
+       COMPLETING Finishing              : Completing    has a list of listeners (promoted once from LIST_*)
+       CANCELLING Finishing              : Cancelling    -- " --
+       FINAL_C    Cancelled              : Cancelled     cancelled (final state)
+       FINAL_F    CompletedExceptionally : Completed     failed for other reason (final state)
+       FINAL_R    <any>                  : Completed     produced some result
 
        === Transitions ===
 
            New states      Active states       Inactive states
 
           +---------+       +---------+                          }
-          | EMPTY_N | --+-> | EMPTY_A | ----+                    } Empty states
-          +---------+   |   +---------+     |                    }
-               |        |     |     ^       |    +----------+
-               |        |     |     |       +--> |  FINAL_* |
-               |        |     V     |       |    +----------+
-               |        |   +---------+     |                    }
-               |        |   | SINGLE  | ----+                    } JobNode states
-               |        |   +---------+     |                    }
-               |        |        |          |                    }
-               |        |        V          |                    }
-               |        |   +---------+     |                    }
-               |        +-- | SINGLE+ | ----+                    }
+          | EMPTY_N | ----> | EMPTY_A | ----+                    } Empty states
+          +---------+       +---------+     |                    }
+               |  |           |     ^       |    +----------+
+               |  |           |     |       +--> |  FINAL_* |
+               |  |           V     |       |    +----------+
+               |  |         +---------+     |                    }
+               |  |         | SINGLE  | ----+                    } JobNode states
+               |  |         +---------+     |                    }
+               |  |              |          |                    }
+               |  |              V          |                    }
+               |  |         +---------+     |                    }
+               |  +-------> | SINGLE+ | ----+                    }
                |            +---------+     |                    }
                |                 |          |
                V                 V          |
           +---------+       +---------+     |                    }
-          | LIST_N  | ----> | LIST_A  | ----+                    } NodeList states
+          | LIST_N  | ----> | LIST_A  | ----+                    } [Inactive]NodeList states
           +---------+       +---------+     |                    }
              |   |             |   |        |
              |   |    +--------+   |        |
@@ -75,7 +75,9 @@
 
 
        This state machine and its transition matrix are optimized for the common case when job is created in active
-       state (EMPTY_A) and at most one completion listener is added to it during its life-time.
+       state (EMPTY_A), at most one completion listener is added to it during its life-time, and it completes
+       successfully without children (in this case it directly goes from EMPTY_A or SINGLE state to FINAL_R
+       state without going to COMPLETING state)
 
        Note, that the actual `_state` variable can also be a reference to atomic operation descriptor `OpDescriptor`
      */
@@ -103,7 +105,7 @@
         @Suppress("DEPRECATION")
         val handle = parent.attachChild(this)
         parentHandle = handle
-        // now check our state _after_ registering (see tryFinalizeStateActually order of actions)
+        // now check our state _after_ registering (see tryFinalizeSimpleState order of actions)
         if (isCompleted) {
             handle.dispose()
             parentHandle = NonDisposableHandle // release it just in case, to aid GC
@@ -139,63 +141,64 @@
 
     public final override val isCompleted: Boolean get() = state !is Incomplete
 
+    public final override val isFailed: Boolean get() {
+        val state = this.state
+        return state is CompletedExceptionally || (state is Finishing && state.isFailing)
+    }
+
     public final override val isCancelled: Boolean get() {
         val state = this.state
-        return state is Cancelled || (state is Finishing && state.cancelled != null)
+        return state is Cancelled || (state is Finishing && state.isCancelling)
     }
 
     // ------------ state update ------------
 
-    /**
-     * Updates current [state] of this job to the final state, invoking all necessary handlers
-     * and/or `on*` methods.
-     *
-     * Returns `false` if current state is not equal to expected.
-     * If this method succeeds, state of this job will never be changed again
-     */
-    private fun tryFinalizeState(expect: Incomplete, proposedUpdate: Any?, mode: Int): Boolean =
-        if (expect is Finishing && expect.cancelled != null) {
-            tryFinalizeCancellingState(expect, proposedUpdate, mode)
-        } else {
-            val update = coerceProposedUpdate(expect, proposedUpdate)
-            tryFinalizeStateActually(expect, update, mode)
+    // Finalizes Finishing -> Completed (terminal state) transition.
+    // ## IMPORTANT INVARIANT: Only one thread can be concurrently invoking this method.
+    private fun tryFinalizeFinishingState(state: Finishing, proposedUpdate: Any?, mode: Int): Boolean {
+        require(proposedUpdate !is Incomplete) // only incomplete -> completed transition is allowed
+        require(this.state === state) // consistency check -- it cannot change
+        require(!state.isSealed) // consistency check -- cannot be sealed yet
+        require(state.isCompleting) // consistency check -- must be marked as completing
+        val proposedException = (proposedUpdate as? CompletedExceptionally)?.cause
+        val proposedCancel = proposedUpdate is Cancelled
+        // Create the final exception and seal the state so that no more exceptions can be added
+        var suppressed = false
+        val finalException = synchronized(state) {
+            val exceptions = state.sealLocked(proposedException)
+            val rootCause = getFinalRootCause(state, exceptions)
+            if (rootCause != null) suppressed = suppressExceptions(rootCause, exceptions)
+            rootCause
         }
-
-    // Finalizes Cancelling -> Cancelled transition
-    private fun tryFinalizeCancellingState(expect: Finishing, proposedUpdate: Any?, mode: Int): Boolean {
-        /*
-         * If job is in 'cancelling' state and we're finalizing job state, we start intricate dance:
-         * 1) Synchronize on state to avoid races with concurrent
-         *    mutations (e.g. when new child is added)
-         * 2) After synchronization check we're still in the expected state
-         * 3) Aggregate final exception under the same lock which protects exceptions
-         *    collection
-         * 4) Pass it upstream
-         */
-        val finalException = synchronized(expect) {
-            if (_state.value !== expect) {
-                return false
-            }
-            if (proposedUpdate is CompletedExceptionally) {
-                expect.addExceptionLocked(proposedUpdate.cause)
-            }
-            /*
-             * Note that new exceptions cannot be added concurrently: state is guarded by lock
-             * and storage is sealed in the end, so all new exceptions will be reported separately
-             */
-            buildException(expect).also { expect.seal() }
+        // Create the final state object
+        val finalState = when {
+            // if we have not failed -> use proposed update value
+            finalException == null -> proposedUpdate
+            // small optimization when we can used proposeUpdate object as is on failure
+            finalException === proposedException && proposedCancel == state.isCancelling -> proposedUpdate
+            // cancelled job final state
+            state.isCancelling -> Cancelled(finalException)
+            // failed job final state
+            else -> CompletedExceptionally(finalException)
         }
-        val update = Cancelled(this, finalException ?: expect.cancelled!!.cause)
-        if (tryFinalizeStateActually(expect, update, mode)) return true
-        //  ^^^^ this CAS never fails: we're in the state when no jobs can be attached, because state is already sealed
-        val error = AssertionError("Unexpected state: ${_state.value}, expected: $expect, update: $update")
-        handleOnCompletionException(error)
-        throw error
+        // Now handle exception
+        if (finalException != null) {
+            if (!failParent(finalException)) handleJobException(finalException)
+        }
+        // Then CAS to completed state -> it must succeed
+        require(_state.compareAndSet(state, finalState)) { "Unexpected state: ${_state.value}, expected: $state, update: $finalState" }
+        // And process all post-completion actions
+        completeStateFinalization(state, finalState, mode, suppressed)
+        return true
     }
 
-    private fun buildException(state: Finishing): Throwable? {
-        val cancelled = state.cancelled!!
-        val suppressed = state.exceptions
+    private fun getFinalRootCause(state: Finishing, exceptions: List<Throwable>): Throwable? {
+        // A case of no exceptions
+        if (exceptions.isEmpty()) {
+            // materialize cancellation exception if it was not materialized yet
+            if (state.isCancelling) return createJobCancellationException()
+            return null
+        }
         /*
          * This is a place where we step on our API limitation:
          * We can't distinguish internal JobCancellationException from our parent
@@ -203,39 +206,45 @@
          *
          * But it has negative consequences: same exception can be added as suppressed more than once.
          * Consider concurrent parent-child relationship:
-         * 1) Child throws E1 and parent throws E2
-         * 2) Parent goes to "Cancelling(E1)" and cancels child with E1
+         * 1) Child throws E1 and parent throws E2.
+         * 2) Parent goes to "Failing(E1)" and cancels child with E1
          * 3) Child goes to "Cancelling(E1)", but throws an exception E2
-         * 4) When child throws, it notifies parent that he is cancelling, adding its exception to parent list of exceptions
-         *    (again, parent don't know whether it's child exception or external exception)
-         * 5) Child builds final exception: E1 with suppressed E2, reports it to parent
+         * 4) When child throws, it notifies parent that it is failing, adding its exception to parent's list of exceptions/
+         * 5) Child builds final exception: E1 with suppressed E2, reports it to parent.
          * 6) Parent aggregates three exceptions: original E1, reported E2 and "final" E1.
          *    It filters the third exception, but adds the second one to the first one, thus adding suppressed duplicate.
          *
-         * Note that it's only happening when both parent and child throw exception simultaneously
+         * Note that it's only happening when both parent and child throw exception simultaneously.
          */
-        var rootCause = cancelled.cause
+        var rootCause = exceptions[0]
         if (rootCause is JobCancellationException) {
             val cause = unwrap(rootCause)
             rootCause = if (cause !== null) {
                 cause
             } else {
-                suppressed.firstOrNull { unwrap(it) != null } ?: return rootCause
-            }
-        }
-        // TODO it should be identity set and optimized for small footprints
-        val seenExceptions = HashSet<Throwable>(suppressed.size)
-        suppressed.forEach {
-            val unwrapped = unwrap(it)
-            if (unwrapped !== null && unwrapped !== rootCause) {
-                if (seenExceptions.add(unwrapped)) {
-                    rootCause.addSuppressedThrowable(unwrapped)
-                }
+                exceptions.firstOrNull { unwrap(it) != null } ?: return rootCause
             }
         }
         return rootCause
     }
 
+    private fun suppressExceptions(rootCause: Throwable, exceptions: List<Throwable>): Boolean {
+        if (exceptions.size <= 1) return false // nothing more to do here
+        // TODO it should be identity set and optimized for small footprints
+        val seenExceptions = HashSet<Throwable>(exceptions.size)
+        var suppressed = false
+        for (i in 1 until exceptions.size) {
+            val unwrapped = unwrap(exceptions[i])
+            if (unwrapped !== null && unwrapped !== rootCause) {
+                if (seenExceptions.add(unwrapped)) {
+                    rootCause.addSuppressedThrowable(unwrapped)
+                    suppressed = true
+                }
+            }
+        }
+        return suppressed
+    }
+
     private tailrec fun unwrap(exception: Throwable): Throwable? =
         if (exception is JobCancellationException) {
             val cause = exception.cause
@@ -244,29 +253,17 @@
             exception
         }
 
-    /**
-     * Tries to actually update [state] of this job to the final state and, if
-     * succeeds, disposes parent handle (detaching child from parent), and
-     * invokes all the handlers to notify on the final state transition.
-     */
-    private fun tryFinalizeStateActually(expect: Incomplete, update: Any?, mode: Int): Boolean {
-        require(update !is Incomplete) // only incomplete -> completed transition is allowed
-
-        /*
-         * We're publishing CompletedExceptionally as OpDescriptor to avoid races with parent:
-         * Job can't report exception before CAS (as it can fail), but after CAS there is a small window
-         * where the parent is considering this job (child) completed, though child has not yet reported its exception.
-         */
-        val updateValue = if (update is CompletedExceptionally) HandleExceptionOp(update) else update
-        if (!_state.compareAndSet(expect, updateValue)) return false // failed
-        if (updateValue is HandleExceptionOp) {
-            updateValue.perform(this) // help perform
-        }
-        completeStateFinalization(expect, update, mode)
+    // fast-path method to finalize normally completed coroutines without children
+    private fun tryFinalizeSimpleState(state: Incomplete, update: Any?, mode: Int): Boolean {
+        check(state !is Finishing) // only for non-finishing state
+        check(update !is CompletedExceptionally) // only for normal completion
+        if (!_state.compareAndSet(state, update)) return false
+        completeStateFinalization(state, update, mode, false)
         return true
     }
 
-    private fun completeStateFinalization(expect: Incomplete, update: Any?, mode: Int) {
+    // suppressed == true when any exceptions were suppressed while building the final completion cause
+    private fun completeStateFinalization(state: Incomplete, update: Any?, mode: Int, suppressed: Boolean) {
         /*
          * Now the job in THE FINAL state. We need to properly handle the resulting state.
          * Order of various invocations here is important.
@@ -277,68 +274,46 @@
             it.dispose() // volatile read parentHandle _after_ state was updated
             parentHandle = NonDisposableHandle // release it just in case, to aid GC
         }
-        val exceptionally = update as? CompletedExceptionally
+        val cause = (update as? CompletedExceptionally)?.cause
         /*
-         * 2) Invoke onCancellationInternal: exception handling, resource cancellation etc.
-         *    Only notify on cancellation once (expect.isCancelling)
+         * 2) Invoke onFailing: for resource cancellation resource cancellation etc.
+         *    Only notify is was not notified yet.
+         *    Note: we do not use notifyFailing here, since we are going to invoke all completion as our next step
          */
-        if (!expect.isCancelling) {
-            onCancellationInternal(exceptionally)
-        }
+        if (!state.isFailing) onFailing(cause)
         /*
          * 3) Invoke completion handlers: .join(), callbacks etc.
          *    It's important to invoke them only AFTER exception handling, see #208
          */
-        val cause = exceptionally?.cause
-        if (expect is JobNode<*>) { // SINGLE/SINGLE+ state -- one completion handler (common case)
+        if (state is JobNode<*>) { // SINGLE/SINGLE+ state -- one completion handler (common case)
             try {
-                expect.invoke(cause)
+                state.invoke(cause)
             } catch (ex: Throwable) {
-                handleOnCompletionException(CompletionHandlerException("Exception in completion handler $expect for $this", ex))
+                handleOnCompletionException(CompletionHandlerException("Exception in completion handler $state for $this", ex))
             }
         } else {
-            expect.list?.notifyCompletion(cause)
+            state.list?.notifyCompletion(cause)
         }
         /*
-         * 4) Invoke onCompletionInternal: onNext(), timeout deregistration etc.
+         * 4) Invoke onCompletionInternal: onNext(), timeout de-registration etc.
          *    It should be last so all callbacks observe consistent state
          *    of the job which doesn't depend on callback scheduling.
          */
-        onCompletionInternal(update, mode)
+        onCompletionInternal(update, mode, suppressed)
     }
 
-    // when Job is in Cancelling state, it can only be promoted to Cancelled state,
-    // so if the proposed Update is not an appropriate Cancelled (preserving the cancellation cause),
-    // then the corresponding Cancelled state is constructed.
-    private fun coerceProposedUpdate(expect: Incomplete, proposedUpdate: Any?): Any? =
-        if (expect is Finishing && expect.cancelled != null && !isCorrespondinglyCancelled(expect.cancelled, proposedUpdate))
-            createCancelled(expect.cancelled, proposedUpdate) else proposedUpdate
-
-    private fun isCorrespondinglyCancelled(cancelled: Cancelled, proposedUpdate: Any?): Boolean {
-        if (proposedUpdate !is Cancelled) return false
-        // NOTE: equality comparison of causes is performed here by design, see equals of JobCancellationException
-        return proposedUpdate.cause == cancelled.cause || proposedUpdate.cause is JobCancellationException
-    }
-
-    private fun createCancelled(cancelled: Cancelled, proposedUpdate: Any?): Cancelled {
-        if (proposedUpdate !is CompletedExceptionally) return cancelled // not exception -- just use original cancelled
-        val exception = proposedUpdate.cause
-        if (cancelled.cause == exception) return cancelled // that is the cancelled we need already!
-        // That could have occurred while coroutine is being cancelled.
-        // Do not spam with JCE in suppressed exceptions
-        if (cancelled.cause !is JobCancellationException) {
-            exception.addSuppressedThrowable(cancelled.cause)
-        }
-        return Cancelled(this, exception)
+    private fun notifyFailing(list: NodeList, cause: Throwable) {
+        // first cancel our own children
+        onFailing(cause)
+        notifyHandlers<JobFailingNode<*>>(list, cause)
+        // then report to the parent that we are failing
+        failParent(cause) // tentative failure report -- does not matter if there is no parent
     }
 
     private fun NodeList.notifyCompletion(cause: Throwable?) =
         notifyHandlers<JobNode<*>>(this, cause)
 
-    private fun notifyCancellation(list: NodeList, cause: Throwable?) =
-        notifyHandlers<JobCancellationNode<*>>(list, cause)
-
-      private inline fun <reified T: JobNode<*>> notifyHandlers(list: NodeList, cause: Throwable?) {
+    private inline fun <reified T: JobNode<*>> notifyHandlers(list: NodeList, cause: Throwable?) {
         var exception: Throwable? = null
         list.forEach<T> { node ->
             try {
@@ -390,13 +365,11 @@
 
     public final override fun getCancellationException(): CancellationException {
         val state = this.state
-        return when {
-            state is Finishing && state.cancelled != null ->
-                state.cancelled.cause.toCancellationException("Job is being cancelled")
-            state is Incomplete ->
-                error("Job was not completed or cancelled yet: $this")
-            state is CompletedExceptionally ->
-                state.cause.toCancellationException("Job has failed")
+        return when (state) {
+            is Finishing -> state.rootCause?.toCancellationException("Job is failing")
+                ?: error("Job is still new or active: $this")
+            is Incomplete -> error("Job is still new or active: $this")
+            is CompletedExceptionally -> state.cause.toCancellationException("Job has failed")
             else -> JobCancellationException("Job has completed normally", null, this)
         }
     }
@@ -408,33 +381,35 @@
      * Returns the cause that signals the completion of this job -- it returns the original
      * [cancel] cause, [JobCancellationException] or **`null` if this job had completed normally**.
      * This function throws [IllegalStateException] when invoked for an job that has not [completed][isCompleted] nor
-     * [isCancelled] yet.
+     * failing yet.
+     *
+     * @suppress **This is unstable API and it is subject to change.**
      */
-    protected fun getCompletionCause(): Throwable? {
-        val state = this.state
-        return when {
-            state is Finishing && state.cancelled != null -> state.cancelled.cause
-            state is Incomplete -> error("Job was not completed or cancelled yet")
-            state is CompletedExceptionally -> state.cause
+    protected fun getCompletionCause(): Throwable? = loopOnState { state ->
+        return when (state) {
+            is Finishing -> state.rootCause
+                ?: error("Job is still new or active: $this")
+            is Incomplete -> error("Job is still new or active: $this")
+            is CompletedExceptionally -> state.cause
             else -> null
         }
     }
 
     @Suppress("OverridingDeprecatedMember")
     public final override fun invokeOnCompletion(handler: CompletionHandler): DisposableHandle =
-        invokeOnCompletion(onCancelling = false, invokeImmediately = true, handler = handler)
+        invokeOnCompletion(onFailing = false, invokeImmediately = true, handler = handler)
 
     @Suppress("OverridingDeprecatedMember")
     public final override fun invokeOnCompletion(handler: CompletionHandler, onCancelling: Boolean): DisposableHandle =
-        invokeOnCompletion(onCancelling = onCancelling, invokeImmediately = true, handler = handler)
+        invokeOnCompletion(onFailing = onCancelling, invokeImmediately = true, handler = handler)
 
     @Suppress("OverridingDeprecatedMember")
     public final override fun invokeOnCompletion(onCancelling_: Boolean, handler: CompletionHandler): DisposableHandle =
-        invokeOnCompletion(onCancelling = onCancelling_, invokeImmediately = true, handler = handler)
+        invokeOnCompletion(onFailing = onCancelling_, invokeImmediately = true, handler = handler)
 
     // todo: non-final as a workaround for KT-21968, should be final in the future
     public override fun invokeOnCompletion(
-        onCancelling: Boolean,
+        onFailing: Boolean,
         invokeImmediately: Boolean,
         handler: CompletionHandler
     ): DisposableHandle {
@@ -444,7 +419,7 @@
                 is Empty -> { // EMPTY_X state -- no completion handlers
                     if (state.isActive) {
                         // try move to SINGLE state
-                        val node = nodeCache ?: makeNode(handler, onCancelling).also { nodeCache = it }
+                        val node = nodeCache ?: makeNode(handler, onFailing).also { nodeCache = it }
                         if (_state.compareAndSet(state, node)) return node
                     } else
                         promoteEmptyToNodeList(state) // that way we can add listener for non-active coroutine
@@ -454,13 +429,33 @@
                     if (list == null) { // SINGLE/SINGLE+
                         promoteSingleToNodeList(state as JobNode<*>)
                     } else {
-                        if (state is Finishing && state.cancelled != null && onCancelling) {
-                            // installing cancellation handler on job that is being cancelled
-                            if (invokeImmediately) handler.invokeIt(state.cancelled.cause)
-                            return NonDisposableHandle
+                        var rootCause: Throwable? = null
+                        var handle: DisposableHandle = NonDisposableHandle
+                        if (onFailing && state is Finishing) {
+                            synchronized(state) {
+                                // check if we are installing failing handler on job that is failing
+                                rootCause = state.rootCause // != null if we are failing
+                                // We add node to the list in two cases --- either the job is not failing
+                                // or we are adding a child to a coroutine that is not completing yet
+                                if (rootCause == null || handler.isHandlerOf<ChildJob>() && !state.isCompleting) {
+                                    // Note: add node the list while holding lock on state (make sure it cannot change)
+                                    val node = nodeCache ?: makeNode(handler, onFailing).also { nodeCache = it }
+                                    if (!addLastAtomic(state, list, node)) return@loopOnState // retry
+                                    // just return node if we don't have to invoke handler (not failing yet)
+                                    if (rootCause == null) return node
+                                    // otherwise handler is invoked immediately out of the synchronized section & handle returned
+                                    handle = node
+                                }
+                            }
                         }
-                        val node = nodeCache ?: makeNode(handler, onCancelling).also { nodeCache = it }
-                        if (addLastAtomic(state, list, node)) return node
+                        if (rootCause != null) {
+                            // Note: attachChild uses invokeImmediately, so it gets invoked when adding to failing job
+                            if (invokeImmediately) handler.invokeIt(rootCause)
+                            return handle
+                        } else {
+                            val node = nodeCache ?: makeNode(handler, onFailing).also { nodeCache = it }
+                            if (addLastAtomic(state, list, node)) return node
+                        }
                     }
                 }
                 else -> { // is complete
@@ -475,10 +470,10 @@
 
     private fun makeNode(handler: CompletionHandler, onCancelling: Boolean): JobNode<*> {
         return if (onCancelling)
-            (handler as? JobCancellationNode<*>)?.also { require(it.job === this) }
-                ?: InvokeOnCancellation(this, handler)
+            (handler as? JobFailingNode<*>)?.also { require(it.job === this) }
+                ?: InvokeOnFailing(this, handler)
         else
-            (handler as? JobNode<*>)?.also { require(it.job === this && it !is JobCancellationNode) }
+            (handler as? JobNode<*>)?.also { require(it.job === this && it !is JobFailingNode) }
                 ?: InvokeOnCompletion(this, handler)
     }
 
@@ -568,96 +563,155 @@
     }
 
     /**
+     * Returns `true` for job that do not have "body block" to complete and should immediately go into
+     * completing state and start waiting for children.
+     *
      * @suppress **This is unstable API and it is subject to change.**
      */
-    internal open val onCancelMode: Int get() = ON_CANCEL_MAKE_CANCELLING
+    internal open val onFailComplete: Boolean get() = false
 
-    public override fun cancel(cause: Throwable?): Boolean = when (onCancelMode) {
-        ON_CANCEL_MAKE_CANCELLING -> makeCancelling(cause)
-        ON_CANCEL_MAKE_COMPLETING -> makeCompleting(Cancelled(this, cause))
-        else -> error("Invalid onCancelMode $onCancelMode")
+    // external cancel with (optional) cause
+    public override fun cancel(cause: Throwable?): Boolean =
+        fail(cause, cancel = true) && handlesException
+
+    // child is reporting failure to the parent
+    public override fun childFailed(cause: Throwable) =
+        fail(cause, cancel = false) && handlesException
+
+    // parent is cancelling child
+    public override fun cancelChild(parentJob: Job) {
+        fail(parentJob, cancel = true)
     }
 
-    // we will be dispatching coroutine to process its cancellation exception, so there is no need for
-    // an extra check for Job status in MODE_CANCELLABLE
-    private fun updateStateCancelled(state: Incomplete, cause: Throwable?) =
-        tryFinalizeState(state, Cancelled(this, cause), mode = MODE_ATOMIC_DEFAULT)
+    // cause is Throwable or Job when cancelChild was invoked, cause can be null only on cancel
+    // returns true is exception was handled, false otherwise
+    private fun fail(cause: Any?, cancel: Boolean): Boolean {
+        if (onFailComplete) {
+            // make sure it is completing, if failMakeCompleting returns true it means it had make it
+            // completing and had recorded exception
+            if (failMakeCompleting(cause, cancel)) return true
+            // otherwise just record failure via makeFailing below
+        }
+        return makeFailing(cause, cancel)
+    }
 
-    // transitions to Cancelling state
-    private fun makeCancelling(cause: Throwable?): Boolean {
+    private fun failMakeCompleting(cause: Any?, cancel: Boolean): Boolean {
         loopOnState { state ->
-            when (state) {
-                is Empty -> { // EMPTY_X state -- no completion handlers
-                    if (state.isActive) {
-                        promoteEmptyToNodeList(state) // this way can wrap it into Cancelling on next pass
-                    } else {
-                        // cancelling a non-started coroutine makes it immediately cancelled
-                        // (and we have no listeners to notify which makes it very simple)
-                        if (updateStateCancelled(state, cause)) return true
-                    }
-                }
-                is JobNode<*> -> { // SINGLE/SINGLE+ state -- one completion handler
-                    promoteSingleToNodeList(state)
-                }
-                is NodeList -> { // LIST -- active list of completion handlers
-                    if (tryMakeCancelling(state, state.list, cause)) return true
-                }
-                is InactiveNodeList -> { // LIST -- inactive list of completion handlers
-                    // cancelling a non-started coroutine makes it immediately cancelled
-                    if (updateStateCancelled(state, cause))
-                        return true
-                }
-                is Finishing -> { // Completing/Cancelling the job, may cancel
-                    if (state.cancelled != null) {
-                        if (cause == null) {
-                            return true
-                        }
-                        /*
-                         * If we failed to add an exception, then `seal` was successfully called
-                         * and `seal` is called only after state update => retry is liveness-safe
-                         */
-                        if (state.addException(cause)) {
-                            return true
-                        } else {
-                            return@loopOnState
-                        }
-                    }
-                    if (tryMakeCancelling(state, state.list, cause)) return true
-                }
-                /*
-                 * Filter out duplicates due to race in the following pattern:
-                 * T1: parent -> completion sequence
-                 * T2: child -> set final state -> signal with final exception to the parent
-                 */
-                is CompletedExceptionally -> return state.cause === cause
-                else -> { // is inactive
-                    return false
-                }
+            if (state !is Incomplete || state is Finishing && state.isCompleting) {
+                return false // already completed/completing, do not even propose update
+            }
+            val proposedUpdate = createFailure(createCauseException(cause), cancel)
+            when (tryMakeCompleting(state, proposedUpdate, mode = MODE_ATOMIC_DEFAULT)) {
+                COMPLETING_ALREADY_COMPLETING -> return false
+                COMPLETING_COMPLETED, COMPLETING_WAITING_CHILDREN -> return true
+                COMPLETING_RETRY -> return@loopOnState
+                else -> error("unexpected result")
             }
         }
     }
 
-    // try make expected state in cancelling on the condition that we're still in this state
-    private fun tryMakeCancelling(expect: Incomplete, list: NodeList, cause: Throwable?): Boolean {
-        val cancelled = Cancelled(this, cause)
-        if (!_state.compareAndSet(expect, Finishing(list, cancelled, false))) return false
-        onFinishingInternal(cancelled)
-        onCancellationInternal(cancelled)
-        // Materialize cause
-        notifyCancellation(list, cancelled.cause)
+    private fun createJobCancellationException() =
+        JobCancellationException("Job was cancelled", null, this)
+
+    // cause is Throwable or Job when cancelChild was invoked, cause can be null only on cancel
+    private fun createCauseException(cause: Any?): Throwable = when(cause) {
+        is Throwable? -> cause ?: createJobCancellationException()
+        else -> (cause as Job).getCancellationException()
+    }
+
+    private fun createFailure(causeException: Throwable, cancel: Boolean): CompletedExceptionally =
+        when {
+            cancel -> Cancelled(causeException)
+            else -> CompletedExceptionally(causeException)
+        }
+
+    // transitions to Failing state
+    // cause is Throwable or Job when cancelChild was invoked, cause can be null only on cancel
+    private fun makeFailing(cause: Any?, cancel: Boolean): Boolean {
+        var causeExceptionCache: Throwable? = null // lazily init result of createCauseException(cause)
+        loopOnState { state ->
+            when (state) {
+                is Finishing -> { // already finishing -- collect exceptions
+                    var notifyRootCause: Throwable? = null
+                    synchronized(state) {
+                        if (state.isSealed) return false // too late, already sealed -- cannot add exception nor mark cancelled
+                        // add exception, do nothing is parent is cancelling child that is already failing
+                        val wasFailing = state.isFailing // will notify if was not failing
+                        // Materialize missing exception if it is the first exception (otherwise -- don't)
+                        if (cause != null || !wasFailing) {
+                            val causeException = causeExceptionCache ?: createCauseException(cause).also { causeExceptionCache = it }
+                            state.addExceptionLocked(causeException)
+                        }
+                        // mark as cancelling if cancel was requested
+                        if (cancel) state.isCancelling = true
+                        // take cause for notification is was not failing before
+                        notifyRootCause = state.rootCause.takeIf { !wasFailing }
+                    }
+                    notifyRootCause?.let { notifyFailing(state.list, it) }
+                    return true
+                }
+                is Incomplete -> {
+                    // Not yet finishing -- try to make it failing
+                    val list = tryPromoteToList(state) ?: return@loopOnState
+                    val causeException = causeExceptionCache ?: createCauseException(cause).also { causeExceptionCache = it }
+                    if (state.isActive) {
+                        // active state becomes failing
+                        if (tryMakeFailing(state, list, causeException, cancel)) return true
+                    } else {
+                        // non active state starts completing
+                        when (tryMakeCompleting(state, createFailure(causeException, cancel), mode = MODE_ATOMIC_DEFAULT)) {
+                            COMPLETING_ALREADY_COMPLETING -> error("Cannot happen in $state")
+                            COMPLETING_COMPLETED, COMPLETING_WAITING_CHILDREN -> return true // ok
+                            COMPLETING_RETRY -> return@loopOnState
+                            else -> error("unexpected result")
+                        }
+                    }
+                }
+                else -> return false // already complete
+            }
+        }
+    }
+
+    // Performs promotion of incomplete coroutine state to NodeList, returns null when need to retry
+    private fun tryPromoteToList(state: Incomplete): NodeList? = state.list ?: null.also {
+        when (state) {
+            is Empty -> promoteEmptyToNodeList(state)
+            is JobNode<*> -> promoteSingleToNodeList(state)
+            else -> error("State should have list: $state")
+        }
+    }
+
+    // try make new failing state on the condition that we're still in the expected state
+    private fun tryMakeFailing(state: Incomplete, list: NodeList, rootCause: Throwable, cancel: Boolean): Boolean {
+        check(state !is Finishing) // only for non-finishing states
+        check(state.isActive) // only for active states
+        // Create failing state (with rootCause!)
+        val failing = Finishing(list, cancel, false, rootCause)
+        if (!_state.compareAndSet(state, failing)) return false
+        // Notify listeners
+        notifyFailing(list, rootCause)
         return true
     }
 
     /**
+     * This function is used by [CompletableDeferred.complete] (and exceptionally) and by [JobImpl.cancel].
+     * It returns `false` on repeated invocation (when this job is already completing).
+     *
      * @suppress **This is unstable API and it is subject to change.**
      */
-    internal fun makeCompleting(proposedUpdate: Any?): Boolean =
-        when (makeCompletingInternal(proposedUpdate, mode = MODE_ATOMIC_DEFAULT)) {
-            COMPLETING_ALREADY_COMPLETING -> false
-            else -> true
+    internal fun makeCompleting(proposedUpdate: Any?): Boolean = loopOnState { state ->
+        when (tryMakeCompleting(state, proposedUpdate, mode = MODE_ATOMIC_DEFAULT)) {
+            COMPLETING_ALREADY_COMPLETING -> return false
+            COMPLETING_COMPLETED, COMPLETING_WAITING_CHILDREN -> return true
+            COMPLETING_RETRY -> return@loopOnState
+            else -> error("unexpected result")
         }
+    }
 
     /**
+     * This function is used by [AbstractCoroutine.resume].
+     * It throws exception on repeated invocation (when this job is already completing).
+     *
      * Returns:
      * * `true` if state was updated to completed/cancelled;
      * * `false` if made completing or it is cancelling and is waiting for children.
@@ -665,58 +719,65 @@
      * @throws IllegalStateException if job is already complete or completing
      * @suppress **This is unstable API and it is subject to change.**
      */
-    internal fun makeCompletingOnce(proposedUpdate: Any?, mode: Int): Boolean =
-        when (makeCompletingInternal(proposedUpdate, mode)) {
-            COMPLETING_COMPLETED -> true
-            COMPLETING_WAITING_CHILDREN -> false
-            else -> throw IllegalStateException("Job $this is already complete or completing, " +
+    internal fun makeCompletingOnce(proposedUpdate: Any?, mode: Int): Boolean = loopOnState { state ->
+        when (tryMakeCompleting(state, proposedUpdate, mode)) {
+            COMPLETING_ALREADY_COMPLETING -> throw IllegalStateException("Job $this is already complete or completing, " +
                 "but is being completed with $proposedUpdate", proposedUpdate.exceptionOrNull)
-        }
-
-    private fun makeCompletingInternal(proposedUpdate: Any?, mode: Int): Int {
-        loopOnState { state ->
-            if (state !is Incomplete)
-                return COMPLETING_ALREADY_COMPLETING
-            if (state is Finishing && state.completing)
-                return COMPLETING_ALREADY_COMPLETING
-            val child: ChildJob? = firstChild(state) ?: // or else complete immediately w/o children
-            when {
-                state !is Finishing && hasOnFinishingHandler(proposedUpdate) -> null // unless it has onFinishing handler
-                tryFinalizeState(state, proposedUpdate, mode) -> return COMPLETING_COMPLETED
-                else -> return@loopOnState
-            }
-            val list = state.list ?: // must promote to list to correctly operate on child lists
-            when (state) {
-                is Empty -> {
-                    promoteEmptyToNodeList(state)
-                    return@loopOnState // retry
-                }
-                is JobNode<*> -> {
-                    promoteSingleToNodeList(state)
-                    return@loopOnState // retry
-                }
-                else -> error("Unexpected state with an empty list: $state")
-            }
-            // cancel all children in list on exceptional completion
-            if (proposedUpdate is CompletedExceptionally)
-                child?.cancelChildrenInternal(proposedUpdate.cause)
-            // switch to completing state
-            val cancelled = (state as? Finishing)?.cancelled ?: (proposedUpdate as? Cancelled)
-            val completing = Finishing(list, cancelled, true)
-            if (_state.compareAndSet(state, completing)) {
-                (state as? Finishing)?.transferExceptions(completing)
-                if (state !is Finishing) onFinishingInternal(proposedUpdate)
-                if (child != null && tryWaitForChild(child, proposedUpdate))
-                    return COMPLETING_WAITING_CHILDREN
-                if (tryFinalizeState(completing, proposedUpdate, mode))
-                    return COMPLETING_COMPLETED
-            }
+            COMPLETING_COMPLETED -> return true
+            COMPLETING_WAITING_CHILDREN -> return false
+            COMPLETING_RETRY -> return@loopOnState
+            else -> error("unexpected result")
         }
     }
 
-    private tailrec fun ChildJob.cancelChildrenInternal(cause: Throwable) {
-        childJob.cancel(JobCancellationException("Child job was cancelled because of parent failure", cause, childJob))
-        nextChild()?.cancelChildrenInternal(cause)
+    private fun tryMakeCompleting(state: Any?, proposedUpdate: Any?, mode: Int): Int {
+        if (state !is Incomplete)
+            return COMPLETING_ALREADY_COMPLETING
+        // find first child
+        val child = firstChild(state)
+        // FAST PATH -- no children to wait for && not finishing && not failing => can complete immediately
+        // Failures always have to go through Finishing state to serialize exception handling
+        if (child == null && state !is Finishing && proposedUpdate !is CompletedExceptionally) {
+            if (!tryFinalizeSimpleState(state, proposedUpdate, mode)) return COMPLETING_RETRY
+            return COMPLETING_COMPLETED
+        }
+        // get state's list or else promote to list to correctly operate on child lists
+        val list = tryPromoteToList(state) ?: return COMPLETING_RETRY
+        // promote to Finishing state if we are not in it yet
+        // This promotion has to be atomic w.r.t to state change, so that a coroutine that is not active yet
+        // atomically transition to finishing & completing state
+        val finishing = state as? Finishing ?: Finishing(list, false, false, null)
+        // must synchronize updates to finishing state
+        var notifyRootCause: Throwable? = null
+        synchronized(finishing) {
+            // check if this state is already completing
+            if (finishing.isCompleting) return COMPLETING_ALREADY_COMPLETING
+            // mark as completing
+            finishing.isCompleting = true
+            // ## IMPORTANT INVARIANT: Only one thread (that had set isCompleting) can go past this point
+            require(!finishing.isSealed) // cannot be sealed
+            // mark as cancelling is the proposed update is to cancel
+            if (proposedUpdate is Cancelled) finishing.isCancelling = true
+            // add new proposed exception to the finishing state
+            val wasFailing = finishing.isFailing
+            (proposedUpdate as? CompletedExceptionally)?.let { finishing.addExceptionLocked(it.cause) }
+            // If it just becomes failing --> must process failing notifications
+            notifyRootCause = finishing.rootCause.takeIf { !wasFailing }
+        }
+        // if we need to promote to finishing then atomically do it here
+        if (finishing !== state) {
+            if (!_state.compareAndSet(state, finishing)) return COMPLETING_RETRY
+        }
+        // process failing notification here -- it cancels all the children _before_ we start to to wait them (sic!!!)
+        notifyRootCause?.let { notifyFailing(list, it) }
+        // now wait for children
+        if (child != null && tryWaitForChild(finishing, child, proposedUpdate))
+            return COMPLETING_WAITING_CHILDREN
+        // otherwise -- we have not children left (all were already cancelled?)
+        if (tryFinalizeFinishingState(finishing, proposedUpdate, mode))
+            return COMPLETING_COMPLETED
+        // otherwise retry
+        return COMPLETING_RETRY
     }
 
     private val Any?.exceptionOrNull: Throwable?
@@ -726,28 +787,26 @@
         state as? ChildJob ?: state.list?.nextChild()
 
     // return false when there is no more incomplete children to wait
-    private tailrec fun tryWaitForChild(child: ChildJob, proposedUpdate: Any?): Boolean {
-        val handle = child.childJob.invokeOnCompletion(invokeImmediately = false,
-            handler = ChildCompletion(this, child, proposedUpdate).asHandler)
+    // ## IMPORTANT INVARIANT: Only one thread can be concurrently invoking this method.
+    private tailrec fun tryWaitForChild(state: Finishing, child: ChildJob, proposedUpdate: Any?): Boolean {
+        val handle = child.childJob.invokeOnCompletion(
+            invokeImmediately = false,
+            handler = ChildCompletion(this, state, child, proposedUpdate).asHandler
+        )
         if (handle !== NonDisposableHandle) return true // child is not complete and we've started waiting for it
         val nextChild = child.nextChild() ?: return false
-        return tryWaitForChild(nextChild, proposedUpdate)
+        return tryWaitForChild(state, nextChild, proposedUpdate)
     }
 
-    /**
-     * @suppress **This is unstable API and it is subject to change.**
-     */
-    internal fun continueCompleting(lastChild: ChildJob, proposedUpdate: Any?) {
-        loopOnState { state ->
-            if (state !is Finishing)
-                throw IllegalStateException("Job $this is found in expected state while completing with $proposedUpdate", proposedUpdate.exceptionOrNull)
-            // figure out if we need to wait for next child
-            val waitChild = lastChild.nextChild()
-            // try wait for next child
-            if (waitChild != null && tryWaitForChild(waitChild, proposedUpdate)) return // waiting for next child
-            // no more children to wait -- try update state
-            if (tryFinalizeState(state, proposedUpdate, MODE_ATOMIC_DEFAULT)) return
-        }
+    // ## IMPORTANT INVARIANT: Only one thread can be concurrently invoking this method.
+    private fun continueCompleting(state: Finishing, lastChild: ChildJob, proposedUpdate: Any?) {
+        require(this.state === state) // consistency check -- it cannot change while we are waiting for children
+        // figure out if we need to wait for next child
+        val waitChild = lastChild.nextChild()
+        // try wait for next child
+        if (waitChild != null && tryWaitForChild(state, waitChild, proposedUpdate)) return // waiting for next child
+        // no more children to wait -- try update state
+        if (tryFinalizeFinishingState(state, proposedUpdate, MODE_ATOMIC_DEFAULT)) return
     }
 
     private fun LockFreeLinkedListNode.nextChild(): ChildJob? {
@@ -772,8 +831,15 @@
     }
 
     @Suppress("OverridingDeprecatedMember")
-    public final override fun attachChild(child: Job): DisposableHandle =
-        invokeOnCompletion(onCancelling = true, handler = ChildJob(this, child).asHandler)
+    public final override fun attachChild(child: Job): DisposableHandle {
+        /*
+         * Note: This function attaches a special ChildNode object. This node object
+         * is handled in a special way on completion on the coroutine (we wait for all of them) and
+         * is handled specially by invokeOnCompletion itself -- it adds this node to the list even
+         * if the job is already failing.
+         */
+        return invokeOnCompletion(onFailing = true, handler = ChildJob(this, child).asHandler)
+    }
 
     @Suppress("OverridingDeprecatedMember")
     public final override fun cancelChildren(cause: Throwable?) {
@@ -790,40 +856,41 @@
     }
 
     /**
-     * This function is invoked once when job is cancelled or is completed.
+     * This function is invoked once when job is failing or is completed.
      * It's an optimization for [invokeOnCompletion] with `onCancelling` set to `true`.
      *
-     * @param exceptionally not null when the the job was cancelled or completed exceptionally,
-     *               null when it has completed normally.
      * @suppress **This is unstable API and it is subject to change.*
      */
-    internal open fun onCancellationInternal(exceptionally: CompletedExceptionally?) {
-        // TODO rename to "onCancelling"
-    }
+    internal open fun onFailing(cause: Throwable?) {}
+
+    // todo: make it for all kinds of coroutines, now only launch & actor override and handleExceptionViaJob
+    internal open fun failParent(exception: Throwable): Boolean = false
 
     /**
-     * Whether job has [onFinishingInternal] handler for given [update]
-     * @suppress **This is unstable API and it is subject to change.**
+     * Returns `true` for jobs that handle their exceptions via [handleJobException] or integrate them
+     * into the job's result via [onCompletionInternal]. The only instance of the [Job] that does not
+     * handle its exceptions is [JobImpl].
+     *
+     * @suppress **This is unstable API and it is subject to change.*
      */
-    internal open fun hasOnFinishingHandler(update: Any?) = false
+    protected open val handlesException: Boolean get() = true
 
     /**
-     * @suppress **This is unstable API and it is subject to change.**
+     * This method is invoked **exactly once** when the final exception of the job is determined
+     * and before it becomes complete. At the moment of invocation the job and all its children are complete.
+     *
+     * @suppress **This is unstable API and it is subject to change.*
      */
-    internal open fun onFinishingInternal(update: Any?) {}
-
-    /**
-     * Method which is invoked once Job becomes [Cancelled] or [CompletedExceptionally].
-     * It's guaranteed that at the moment of invocation the job and all its children are complete.
-     */
-    internal open fun handleJobException(exception: Throwable) {}
+    protected open fun handleJobException(exception: Throwable) {}
 
     /**
      * Override for post-completion actions that need to do something with the state.
+     * @param state the final state.
      * @param mode completion mode.
+     * @param suppressed true when any exceptions were suppressed while building the final completion cause.
      * @suppress **This is unstable API and it is subject to change.**
      */
-    internal open fun onCompletionInternal(state: Any?, mode: Int) {}
+    internal open fun onCompletionInternal(state: Any?, mode: Int, suppressed: Boolean) {}
 
     // for nicer debugging
     public override fun toString(): String =
@@ -838,89 +905,106 @@
         val state = this.state
         return when (state) {
             is Finishing -> buildString {
-                if (state.cancelled != null) append("Cancelling")
-                if (state.completing) append("Completing")
+                when { // cancelling implies failing
+                    state.isCancelling -> append("Cancelling")
+                    state.isFailing -> append("Failing")
+                    else -> append("Active")
+                }
+                if (state.isCompleting) append("Completing")
             }
             is Incomplete -> if (state.isActive) "Active" else "New"
             is Cancelled -> "Cancelled"
-            is CompletedExceptionally -> "CompletedExceptionally"
+            is CompletedExceptionally -> "Failed"
             else -> "Completed"
         }
     }
 
-    // Cancelling or Completing
+    // Completing, Failing, Cancelling states,
+    // All updates are guarded by synchronized(this), reads are volatile
     @Suppress("UNCHECKED_CAST")
     private class Finishing(
         override val list: NodeList,
-        @JvmField val cancelled: Cancelled?, /* != null when cancelling */
-        @JvmField val completing: Boolean /* true when completing */
+        @Volatile
+        @JvmField var isCancelling: Boolean,
+        @Volatile
+        @JvmField var isCompleting: Boolean,
+        @Volatile
+        @JvmField var rootCause: Throwable? // NOTE: rootCause is kept even when SEALED
     ) : SynchronizedObject(), Incomplete {
-        override val isActive: Boolean get() = cancelled == null
+        @Volatile
+        private var _exceptionsHolder: Any? = null // Contains null | Throwable | ArrayList | SEALED
 
+        // NotE: cannot be modified when sealed
+        val isSealed: Boolean get() = _exceptionsHolder === SEALED
+        val isFailing: Boolean get() = rootCause != null
+        override val isActive: Boolean get() = !isFailing
+
+        // Seals current state and returns list of exceptions
         // guarded by `synchronized(this)`
-        val exceptions: List<Throwable> get() = when(_exceptionsHolder) {
-            NOT_INITIALIZED -> emptyList()
-            is Throwable -> listOf(_exceptionsHolder as Throwable) // EA should handle this
-            else -> (_exceptionsHolder as List<Throwable>)
+        fun sealLocked(proposedException: Throwable?): List<Throwable> {
+            val eh = _exceptionsHolder // volatile read
+            val list = when(eh) {
+                null -> allocateList()
+                is Throwable -> allocateList().also { it.add(eh) }
+                is ArrayList<*> -> eh as ArrayList<Throwable>
+                else -> error("State is $eh") // already sealed -- cannot happen
+            }
+            val rootCause = this.rootCause // volatile read
+            rootCause?.let { list.add(0, it) } // note -- rootCause goes to the beginning
+            if (proposedException != null && proposedException != rootCause) list.add(proposedException)
+            _exceptionsHolder = SEALED
+            return list
         }
 
         // guarded by `synchronized(this)`
-        // Contains null | NOT_INITIALIZED | Throwable | ArrayList
-        private var _exceptionsHolder: Any? = if (cancelled == null) null else NOT_INITIALIZED
-
-        fun addException(exception: Throwable): Boolean =
-            synchronized(this) {
-                addExceptionLocked(exception)
+        fun addExceptionLocked(exception: Throwable) {
+            val rootCause = this.rootCause // volatile read
+            if (rootCause == null) {
+                this.rootCause = exception
+                return
             }
-
-        // guarded by `synchronized(this)`
-        fun addExceptionLocked(exception: Throwable): Boolean =
-            when (_exceptionsHolder) {
-                null -> false
-                NOT_INITIALIZED -> {
-                    _exceptionsHolder = exception
-                    true
-                }
+            if (exception === rootCause) return // nothing to do
+            val eh = _exceptionsHolder // volatile read
+            when (eh) {
+                null -> _exceptionsHolder = exception
                 is Throwable -> {
-                    val previous = _exceptionsHolder
-                    _exceptionsHolder = ArrayList<Any?>(4).apply {
-                        add(previous)
+                    if (exception === eh) return // nothing to do
+                    _exceptionsHolder = allocateList().apply {
+                        add(eh)
                         add(exception)
 
                     }
-                    true
                 }
-                else -> (_exceptionsHolder as ArrayList<Throwable>).add(exception)
-            }
-
-        /**
-         * Seals current state. After [seal] call all consecutive calls to [addException]
-         * return `false` forcing callers to handle pending exception by themselves.
-         * This call should be guarded by `synchronized(this)`
-         */
-        fun seal() {
-            _exceptionsHolder = null
-        }
-
-        fun transferExceptions(to: Finishing) {
-            synchronized(this) {
-                synchronized(to) {
-                    val holder = _exceptionsHolder
-                    when(holder) {
-                        // Note: "to" cannot be sealed at this point and adding exception there mush succeed.
-                        is Throwable -> require(to.addExceptionLocked(holder))
-                        is List<*> -> holder.forEach {
-                            require(to.addExceptionLocked(it as Throwable))
-                        }
-                    }
-                    seal()
-                }
+                is ArrayList<*> -> (eh as ArrayList<Throwable>).add(exception)
+                else -> error("State is $eh") // already sealed -- cannot happen
             }
         }
+
+        private fun allocateList() = ArrayList<Throwable>(4)
+
+        override fun toString(): String =
+            "Finishing[cancelling=$isCancelling, completing=$isCompleting, rootCause=$rootCause, exceptions=$_exceptionsHolder, list=$list]"
     }
 
+    private val Incomplete.isFailing: Boolean
+        get() = this is Finishing && isFailing
+
     private val Incomplete.isCancelling: Boolean
-        get() = this is Finishing && cancelled != null
+        get() = this is Finishing && isCancelling
+
+    // Used by parent that is waiting for child completion
+    private class ChildCompletion(
+        private val parent: JobSupport,
+        private val state: Finishing,
+        private val child: ChildJob,
+        private val proposedUpdate: Any?
+    ) : JobNode<Job>(child.childJob) {
+        override fun invoke(cause: Throwable?) {
+            parent.continueCompleting(state, child, proposedUpdate)
+        }
+        override fun toString(): String =
+            "ChildCompletion[$child, $proposedUpdate]"
+    }
 
     /*
      * =================================================================================================
@@ -1017,34 +1101,20 @@
         else
             block.startCoroutineCancellable(state as T, select.completion)
     }
-
-    class HandleExceptionOp(val original: CompletedExceptionally) : OpDescriptor() {
-
-        override fun perform(affected: Any?): Any? {
-            val job = (affected as JobSupport)
-            if (job._state.compareAndSet(this, original)) {
-                job.handleJobException(original.cause)
-            }
-
-            return null
-        }
-    }
 }
 
 // --------------- helper classes & constants for job implementation
 
-internal const val ON_CANCEL_MAKE_CANCELLING = 0
-internal const val ON_CANCEL_MAKE_COMPLETING = 1
-
 private const val COMPLETING_ALREADY_COMPLETING = 0
 private const val COMPLETING_COMPLETED = 1
 private const val COMPLETING_WAITING_CHILDREN = 2
+private const val COMPLETING_RETRY = 3
 
 private const val RETRY = -1
 private const val FALSE = 0
 private const val TRUE = 1
 
-private val NOT_INITIALIZED = Symbol("NOT_INITIALIZED")
+private val SEALED = Symbol("SEALED")
 
 private val EMPTY_NEW = Empty(false)
 private val EMPTY_ACTIVE = Empty(true)
@@ -1056,13 +1126,8 @@
 
 internal class JobImpl(parent: Job? = null) : JobSupport(true) {
     init { initParentJobInternal(parent) }
-    override val onCancelMode: Int get() = ON_CANCEL_MAKE_COMPLETING
-
-    override fun cancel(cause: Throwable?): Boolean {
-        // JobImpl can't handle an exception, thus always returns false
-        super.cancel(cause)
-        return false
-    }
+    override val onFailComplete get() = true
+    override val handlesException: Boolean get() = false
 }
 
 // -------- invokeOnCompletion nodes
@@ -1157,31 +1222,28 @@
 // -------- invokeOnCancellation nodes
 
 /**
- * Marker for node that shall be invoked on cancellation (in _cancelling_ state).
- * **Note: may be invoked multiple times during cancellation.**
+ * Marker for node that shall be invoked on in _failing_ state.
+ * **Note: may be invoked multiple times.**
  */
-internal abstract class JobCancellationNode<out J : Job>(job: J) : JobNode<J>(job)
+internal abstract class JobFailingNode<out J : Job>(job: J) : JobNode<J>(job)
 
-private class InvokeOnCancellation(
+private class InvokeOnFailing(
     job: Job,
     private val handler: CompletionHandler
-) : JobCancellationNode<Job>(job)  {
+) : JobFailingNode<Job>(job)  {
     // delegate handler shall be invoked at most once, so here is an additional flag
-    private val _invoked = atomic(0)
+    private val _invoked = atomic(0) // todo: replace with atomic boolean after migration to recent atomicFu
     override fun invoke(cause: Throwable?) {
         if (_invoked.compareAndSet(0, 1)) handler.invoke(cause)
     }
-    override fun toString() = "InvokeOnCancellation[$classSimpleName@$hexAddress]"
+    override fun toString() = "InvokeOnFailing[$classSimpleName@$hexAddress]"
 }
 
 internal class ChildJob(
     parent: JobSupport,
     @JvmField val childJob: Job
-) : JobCancellationNode<JobSupport>(parent) {
-    override fun invoke(cause: Throwable?) {
-        // Always materialize the actual instance of parent's completion exception and cancel child with it
-        childJob.cancel(job.getCancellationException())
-    }
+) : JobFailingNode<JobSupport>(parent) {
+    override fun invoke(cause: Throwable?) = childJob.cancelChild(job)
     override fun toString(): String = "ChildJob[$childJob]"
 }
 
@@ -1189,19 +1251,11 @@
 internal class ChildContinuation(
     parent: Job,
     @JvmField val child: AbstractContinuation<*>
-) : JobCancellationNode<Job>(parent) {
+) : JobFailingNode<Job>(parent) {
     override fun invoke(cause: Throwable?) {
         child.cancel(job.getCancellationException())
     }
-    override fun toString(): String = "ChildContinuation[$child]"
+    override fun toString(): String =
+        "ChildContinuation[$child]"
 }
 
-private class ChildCompletion(
-    private val parent: JobSupport,
-    private val child: ChildJob,
-    private val proposedUpdate: Any?
-) : JobNode<Job>(child.childJob) {
-    override fun invoke(cause: Throwable?) {
-        parent.continueCompleting(child, proposedUpdate)
-    }
-}
diff --git a/common/kotlinx-coroutines-core-common/src/NonCancellable.kt b/common/kotlinx-coroutines-core-common/src/NonCancellable.kt
index 1356a93..e6518bb 100644
--- a/common/kotlinx-coroutines-core-common/src/NonCancellable.kt
+++ b/common/kotlinx-coroutines-core-common/src/NonCancellable.kt
@@ -24,6 +24,9 @@
     override val isActive: Boolean  get() = true
 
     /** Always returns `false`. */
+    override val isFailed: Boolean get() = false
+
+    /** Always returns `false`. */
     override val isCompleted: Boolean get() = false
 
     /** Always returns `false`. */
@@ -59,11 +62,17 @@
         NonDisposableHandle
 
     /** Always returns [NonDisposableHandle]. */
-    override fun invokeOnCompletion(onCancelling: Boolean, invokeImmediately: Boolean, handler: CompletionHandler): DisposableHandle =
+    override fun invokeOnCompletion(onFailing: Boolean, invokeImmediately: Boolean, handler: CompletionHandler): DisposableHandle =
         NonDisposableHandle
 
     /** Always returns `false`. */
-    override fun cancel(cause: Throwable?): Boolean = false
+    override fun cancel(cause: Throwable?): Boolean = false // never handles exceptions
+
+    /** @suppress */
+    override fun childFailed(cause: Throwable): Boolean = false // never handles exceptions
+
+    /** @suppress */
+    override fun cancelChild(parentJob: Job): Unit = error("Cannot be invoked, does not have a parent")
 
     /** Always returns [emptySequence]. */
     override val children: Sequence<Job>
diff --git a/common/kotlinx-coroutines-core-common/src/Scheduled.kt b/common/kotlinx-coroutines-core-common/src/Scheduled.kt
index baaa9cd..a99bb9b 100644
--- a/common/kotlinx-coroutines-core-common/src/Scheduled.kt
+++ b/common/kotlinx-coroutines-core-common/src/Scheduled.kt
@@ -82,7 +82,7 @@
     }
 
     @Suppress("UNCHECKED_CAST")
-    internal override fun onCompletionInternal(state: Any?, mode: Int) {
+    internal override fun onCompletionInternal(state: Any?, mode: Int, suppressed: Boolean) {
         if (state is CompletedExceptionally)
             uCont.resumeUninterceptedWithExceptionMode(state.cause, mode)
         else
diff --git a/common/kotlinx-coroutines-core-common/src/channels/Broadcast.kt b/common/kotlinx-coroutines-core-common/src/channels/Broadcast.kt
index c35e39a..4623ac1 100644
--- a/common/kotlinx-coroutines-core-common/src/channels/Broadcast.kt
+++ b/common/kotlinx-coroutines-core-common/src/channels/Broadcast.kt
@@ -111,16 +111,21 @@
     override val channel: SendChannel<E>
         get() = this
 
-    public override fun cancel(cause: Throwable?): Boolean = super.cancel(cause)
+    override fun cancel(cause: Throwable?): Boolean {
+        val wasCancelled = _channel.cancel(cause)
+        if (wasCancelled) super.cancel(cause) // cancel the job
+        return wasCancelled
+    }
 
-    override fun onCancellationInternal(exceptionally: CompletedExceptionally?) {
-        val cause = exceptionally?.cause
-        val processed = when (exceptionally) {
-            is Cancelled -> _channel.cancel(cause) // producer coroutine was cancelled -- cancel channel
-            else -> _channel.close(cause) // producer coroutine has completed -- close channel
+    override fun onCompletionInternal(state: Any?, mode: Int, suppressed: Boolean) {
+        val cause = (state as? CompletedExceptionally)?.cause
+        val processed = when (state) {
+            // producer coroutine was cancelled -- cancel channel, but without cause if it was closed without cause
+            is Cancelled -> _channel.cancel(state.cause)
+            // producer coroutine completed normally or exceptionally -- close channel with cause
+            else -> _channel.close(cause)
         }
-        if (!processed && cause != null)
-            handleCoroutineException(context, cause, this)
+        if (cause != null && !processed && suppressed) handleExceptionViaHandler(context, cause)
     }
 }
 
diff --git a/common/kotlinx-coroutines-core-common/src/channels/ChannelCoroutine.kt b/common/kotlinx-coroutines-core-common/src/channels/ChannelCoroutine.kt
index 598063e..d2752d4 100644
--- a/common/kotlinx-coroutines-core-common/src/channels/ChannelCoroutine.kt
+++ b/common/kotlinx-coroutines-core-common/src/channels/ChannelCoroutine.kt
@@ -14,5 +14,9 @@
 ) : AbstractCoroutine<Unit>(parentContext, active), Channel<E> by _channel {
     val channel: Channel<E> get() = this
 
-    override fun cancel(cause: Throwable?): Boolean = super.cancel(cause)
+    override fun cancel(cause: Throwable?): Boolean {
+        val wasCancelled = _channel.cancel(cause)
+        if (wasCancelled) super.cancel(cause) // cancel the job
+        return wasCancelled
+    }
 }
diff --git a/common/kotlinx-coroutines-core-common/src/channels/Produce.kt b/common/kotlinx-coroutines-core-common/src/channels/Produce.kt
index 9f234aa..6a77063 100644
--- a/common/kotlinx-coroutines-core-common/src/channels/Produce.kt
+++ b/common/kotlinx-coroutines-core-common/src/channels/Produce.kt
@@ -148,18 +148,17 @@
 private class ProducerCoroutine<E>(
     parentContext: CoroutineContext, channel: Channel<E>
 ) : ChannelCoroutine<E>(parentContext, channel, active = true), ProducerScope<E>, ProducerJob<E> {
-
     override val isActive: Boolean
         get() = super<ChannelCoroutine>.isActive
 
-    override fun onCancellationInternal(exceptionally: CompletedExceptionally?) {
-        val cause = exceptionally?.cause
-        val processed = when (exceptionally) {
+    override fun onCompletionInternal(state: Any?, mode: Int, suppressed: Boolean) {
+        val cause = (state as? CompletedExceptionally)?.cause
+        val processed = when (state) {
             // producer coroutine was cancelled -- cancel channel, but without cause if it was closed without cause
-            is Cancelled -> _channel.cancel(if (cause is CancellationException) null else cause)
-            else -> _channel.close(cause) // producer coroutine has completed -- close channel
+            is Cancelled -> _channel.cancel(state.cause)
+            // producer coroutine completed normally or exceptionally -- close channel with cause
+            else -> _channel.close(cause)
         }
-        if (!processed && cause != null)
-            handleCoroutineException(context, cause, this)
+        if (cause != null && !processed && suppressed) handleExceptionViaHandler(context, cause)
     }
 }
diff --git a/common/kotlinx-coroutines-core-common/src/internal/Scopes.kt b/common/kotlinx-coroutines-core-common/src/internal/Scopes.kt
index e2e7475..d087bc8 100644
--- a/common/kotlinx-coroutines-core-common/src/internal/Scopes.kt
+++ b/common/kotlinx-coroutines-core-common/src/internal/Scopes.kt
@@ -7,20 +7,13 @@
 import kotlinx.coroutines.experimental.*
 import kotlin.coroutines.experimental.*
 
-internal class ScopeOwnerCoroutine<R>(
+/**
+ * This is a coroutine instance that is created by [coroutineScope] builder.
+ * It is just a vanilla instance of [AbstractCoroutine].
+ */
+internal class ScopeCoroutine<R>(
     parentContext: CoroutineContext
-) : AbstractCoroutine<R>(parentContext, true), CoroutineScope {
-
-    override val coroutineContext: CoroutineContext = parentContext + this
-
-    /*
-     * Always return true, so final exception is in the scope before its completion.
-     */
-    override fun cancel(cause: Throwable?): Boolean {
-        super.cancel(cause)
-        return true
-    }
-}
+) : AbstractCoroutine<R>(parentContext, true)
 
 internal class ContextScope(context: CoroutineContext) : CoroutineScope {
     override val coroutineContext: CoroutineContext = context
diff --git a/common/kotlinx-coroutines-core-common/src/selects/Select.kt b/common/kotlinx-coroutines-core-common/src/selects/Select.kt
index 23b6752..5a2208c 100644
--- a/common/kotlinx-coroutines-core-common/src/selects/Select.kt
+++ b/common/kotlinx-coroutines-core-common/src/selects/Select.kt
@@ -273,19 +273,19 @@
     private fun initCancellability() {
         val parent = context[Job] ?: return
         val newRegistration = parent.invokeOnCompletion(
-            onCancelling = true, handler = SelectOnCancellation(parent).asHandler)
+            onFailing = true, handler = SelectOnFailing(parent).asHandler)
         parentHandle = newRegistration
         // now check our state _after_ registering
         if (isSelected) newRegistration.dispose()
     }
 
-    private inner class SelectOnCancellation(job: Job) : JobCancellationNode<Job>(job) {
+    private inner class SelectOnFailing(job: Job) : JobFailingNode<Job>(job) {
         // Note: may be invoked multiple times, but only the first trySelect succeeds anyway
         override fun invoke(cause: Throwable?) {
             if (trySelect(null))
                 resumeSelectCancellableWithException(job.getCancellationException())
         }
-        override fun toString(): String = "SelectOnCancellation[${this@SelectBuilderImpl}]"
+        override fun toString(): String = "SelectOnFailing[${this@SelectBuilderImpl}]"
     }
 
     private val state: Any? get() {
diff --git a/common/kotlinx-coroutines-core-common/test/AbstractCoroutineTest.kt b/common/kotlinx-coroutines-core-common/test/AbstractCoroutineTest.kt
index 80a5705..45ce2a4 100644
--- a/common/kotlinx-coroutines-core-common/test/AbstractCoroutineTest.kt
+++ b/common/kotlinx-coroutines-core-common/test/AbstractCoroutineTest.kt
@@ -4,7 +4,6 @@
 
 package kotlinx.coroutines.experimental
 
-import kotlin.coroutines.experimental.*
 import kotlin.test.*
 
 class AbstractCoroutineTest : TestBase() {
@@ -33,7 +32,7 @@
             }
         }
 
-        coroutine.invokeOnCompletion(onCancelling = true) {
+        coroutine.invokeOnCompletion(onFailing = true) {
             assertTrue(it == null)
             expect(6)
         }
@@ -73,7 +72,7 @@
             }
         }
 
-        coroutine.invokeOnCompletion(onCancelling = true) {
+        coroutine.invokeOnCompletion(onFailing = true) {
             assertTrue(it is TestException0)
             expect(6)
         }
diff --git a/common/kotlinx-coroutines-core-common/test/AsyncLazyTest.kt b/common/kotlinx-coroutines-core-common/test/AsyncLazyTest.kt
index 2278ae0..8c79ea9 100644
--- a/common/kotlinx-coroutines-core-common/test/AsyncLazyTest.kt
+++ b/common/kotlinx-coroutines-core-common/test/AsyncLazyTest.kt
@@ -19,10 +19,10 @@
         }
         expect(2)
         assertTrue(!d.isActive && !d.isCompleted)
-        assertTrue(d.await() == 42)
+        assertEquals(d.await(), 42)
         assertTrue(!d.isActive && d.isCompleted && !d.isCompletedExceptionally)
         expect(4)
-        assertTrue(d.await() == 42) // second await -- same result
+        assertEquals(d.await(), 42) // second await -- same result
         finish(5)
     }
 
@@ -37,10 +37,10 @@
         }
         expect(2)
         assertTrue(!d.isActive && !d.isCompleted)
-        assertTrue(d.await() == 42)
+        assertEquals(d.await(), 42)
         assertTrue(!d.isActive && d.isCompleted && !d.isCompletedExceptionally)
         expect(5)
-        assertTrue(d.await() == 42) // second await -- same result
+        assertEquals(d.await(), 42) // second await -- same result
         finish(6)
     }
 
@@ -66,7 +66,7 @@
         yield() // yield to second child (lazy async is not computing yet)
         expect(5)
         assertTrue(!d.isActive && !d.isCompleted)
-        assertTrue(d.await() == 42) // starts computing
+        assertEquals(d.await(), 42) // starts computing
         assertTrue(!d.isActive && d.isCompleted && !d.isCompletedExceptionally)
         finish(8)
     }
@@ -135,7 +135,7 @@
         yield() // yield to started coroutine
         assertTrue(!d.isActive && d.isCompleted && !d.isCompletedExceptionally) // and it finishes
         expect(5)
-        assertTrue(d.await() == 42) // await sees result
+        assertEquals(d.await(), 42) // await sees result
         finish(6)
     }
 
@@ -155,7 +155,7 @@
         assertTrue(!d.cancel())
         assertTrue(!d.start())
         finish(3)
-        assertTrue(d.await() == 42) // await shall throw CancellationException
+        assertEquals(d.await(), 42) // await shall throw CancellationException
         expectUnreached()
     }
 
@@ -183,7 +183,7 @@
         assertTrue(d.cancel())
         assertTrue(!d.isActive && !d.isCompletedExceptionally && d.isCancelled) // still cancelling
         finish(6)
-        assertTrue(d.await() == 42) // await shall throw CancellationException
+        assertEquals(d.await(), 42) // await shall throw CancellationException
         expectUnreached()
     }
 
diff --git a/common/kotlinx-coroutines-core-common/test/CoroutineExceptionHandlerTest.kt b/common/kotlinx-coroutines-core-common/test/CoroutineExceptionHandlerTest.kt
index 366f1e4..71883ec 100644
--- a/common/kotlinx-coroutines-core-common/test/CoroutineExceptionHandlerTest.kt
+++ b/common/kotlinx-coroutines-core-common/test/CoroutineExceptionHandlerTest.kt
@@ -7,23 +7,42 @@
 import kotlin.test.*
 
 class CoroutineExceptionHandlerTest : TestBase() {
+    // Parent Job() does not handle exception --> handler is invoked on child crash
     @Test
-    fun testCoroutineExceptionHandlerCreator() = runTest {
+    fun testJob() = runTest {
         expect(1)
         var coroutineException: Throwable? = null
         val handler = CoroutineExceptionHandler { _, ex ->
             coroutineException = ex
             expect(3)
         }
-
-        val job = launch(handler + Job()) {
+        val parent = Job()
+        val job = launch(handler + parent) {
             throw TestException()
         }
-
         expect(2)
         job.join()
         finish(4)
         assertTrue(coroutineException is TestException)
+        assertTrue(parent.isFailed)
+    }
+
+    // Parent CompletableDeferred() "handles" exception --> handler is NOT invoked on child crash
+    @Test
+    fun testCompletableDeferred() = runTest {
+        expect(1)
+        val handler = CoroutineExceptionHandler { _, ex ->
+            expectUnreached()
+        }
+        val parent = CompletableDeferred<Unit>()
+        val job = launch(handler + parent) {
+            throw TestException()
+        }
+        expect(2)
+        job.join()
+        finish(3)
+        assertTrue(parent.isFailed)
+        assertTrue(parent.getCompletionExceptionOrNull() is TestException)
     }
 
     private class TestException: RuntimeException()
diff --git a/common/kotlinx-coroutines-core-common/test/FailedJobTest.kt b/common/kotlinx-coroutines-core-common/test/FailedJobTest.kt
new file mode 100644
index 0000000..d90563c
--- /dev/null
+++ b/common/kotlinx-coroutines-core-common/test/FailedJobTest.kt
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2016-2018 JetBrains s.r.o. Use of this source code is governed by the Apache 2.0 license.
+ */
+
+@file:Suppress("NAMED_ARGUMENTS_NOT_ALLOWED") // KT-21913
+
+package kotlinx.coroutines.experimental
+
+import kotlin.test.*
+
+// see https://github.com/Kotlin/kotlinx.coroutines/issues/585
+class FailedJobTest : TestBase() {
+    @Test
+    fun testCancelledJob() = runTest {
+        expect(1)
+        val job = launch {
+            expectUnreached()
+        }
+        expect(2)
+        job.cancelAndJoin()
+        finish(3)
+        assertTrue(job.isCompleted)
+        assertTrue(!job.isActive)
+        assertTrue(job.isCancelled)
+    }
+
+    @Test
+    fun testFailedJob() = runTest(
+        unhandled = listOf({it -> it is TestException })
+    ) {
+        expect(1)
+        val job = launch(NonCancellable) {
+            expect(3)
+            throw TestException()
+        }
+        expect(2)
+        job.join()
+        finish(4)
+        assertTrue(job.isCompleted)
+        assertTrue(!job.isActive)
+        assertTrue(!job.isCancelled)
+    }
+
+    @Test
+    fun testFailedChildJob() = runTest(
+        unhandled = listOf({it -> it is TestException })
+    ) {
+        expect(1)
+        val job = launch(NonCancellable) {
+            expect(3)
+            launch {
+                throw TestException()
+            }
+        }
+        expect(2)
+        job.join()
+        finish(4)
+        assertTrue(job.isCompleted)
+        assertTrue(!job.isActive)
+        assertTrue(!job.isCancelled)
+    }
+
+    private class TestException : Exception()
+}
\ No newline at end of file
diff --git a/common/kotlinx-coroutines-core-common/test/JobStatesTest.kt b/common/kotlinx-coroutines-core-common/test/JobStatesTest.kt
new file mode 100644
index 0000000..df0fad6
--- /dev/null
+++ b/common/kotlinx-coroutines-core-common/test/JobStatesTest.kt
@@ -0,0 +1,181 @@
+/*
+ * Copyright 2016-2018 JetBrains s.r.o. Use of this source code is governed by the Apache 2.0 license.
+ */
+
+@file:Suppress("NAMED_ARGUMENTS_NOT_ALLOWED") // KT-21913
+
+package kotlinx.coroutines.experimental
+
+import kotlin.test.*
+
+/**
+ * Tests that the transitions to the state of the [Job] correspond to documentation in the
+ * table that is presented in the [Job] documentation.
+ */
+class JobStatesTest : TestBase() {
+    @Test
+    public fun testNormalCompletion() = runTest {
+        expect(1)
+        val job = launch(start = CoroutineStart.LAZY) {
+            expect(2)
+            // launches child
+            launch {
+                expect(4)
+            }
+            // completes normally
+        }
+        // New job
+        assertFalse(job.isActive)
+        assertFalse(job.isCompleted)
+        assertFalse(job.isFailed)
+        assertFalse(job.isCancelled)
+        // New -> Active
+        job.start()
+        assertTrue(job.isActive)
+        assertFalse(job.isCompleted)
+        assertFalse(job.isFailed)
+        assertFalse(job.isCancelled)
+        // Active -> Completing
+        yield() // scheduled & starts child
+        expect(3)
+        assertTrue(job.isActive)
+        assertFalse(job.isCompleted)
+        assertFalse(job.isFailed)
+        assertFalse(job.isCancelled)
+        // Completing -> Completed
+        yield()
+        finish(5)
+        assertFalse(job.isActive)
+        assertTrue(job.isCompleted)
+        assertFalse(job.isFailed)
+        assertFalse(job.isCancelled)
+    }
+
+    @Test
+    public fun testCompletingFailed() = runTest(
+        unhandled = listOf({ it -> it is TestException })
+    ) {
+        expect(1)
+        val job = launch(NonCancellable, start = CoroutineStart.LAZY) {
+            expect(2)
+            // launches child
+            launch {
+                expect(4)
+                throw TestException()
+            }
+            // completes normally
+        }
+        // New job
+        assertFalse(job.isActive)
+        assertFalse(job.isCompleted)
+        assertFalse(job.isFailed)
+        assertFalse(job.isCancelled)
+        // New -> Active
+        job.start()
+        assertTrue(job.isActive)
+        assertFalse(job.isCompleted)
+        assertFalse(job.isFailed)
+        assertFalse(job.isCancelled)
+        // Active -> Completing
+        yield() // scheduled & starts child
+        expect(3)
+        assertTrue(job.isActive)
+        assertFalse(job.isCompleted)
+        assertFalse(job.isFailed)
+        assertFalse(job.isCancelled)
+        // Completing -> Failed
+        yield()
+        finish(5)
+        assertFalse(job.isActive)
+        assertTrue(job.isCompleted)
+        assertTrue(job.isFailed)
+        assertFalse(job.isCancelled)
+    }
+
+    @Test
+    public fun testFailing() = runTest(
+        unhandled = listOf({ it -> it is TestException })
+    ) {
+        expect(1)
+        val job = launch(NonCancellable, start = CoroutineStart.LAZY) {
+            expect(2)
+            // launches child
+            launch(start = CoroutineStart.ATOMIC) {
+                expect(4)
+            }
+            // failing
+            throw TestException()
+        }
+        // New job
+        assertFalse(job.isActive)
+        assertFalse(job.isCompleted)
+        assertFalse(job.isFailed)
+        assertFalse(job.isCancelled)
+        // New -> Active
+        job.start()
+        assertTrue(job.isActive)
+        assertFalse(job.isCompleted)
+        assertFalse(job.isFailed)
+        assertFalse(job.isCancelled)
+        // Active -> Failing
+        yield() // scheduled & starts child
+        expect(3)
+        assertFalse(job.isActive)
+        assertFalse(job.isCompleted)
+        assertTrue(job.isFailed)
+        assertFalse(job.isCancelled)
+        // Failing -> Failed
+        yield()
+        finish(5)
+        assertFalse(job.isActive)
+        assertTrue(job.isCompleted)
+        assertTrue(job.isFailed)
+        assertFalse(job.isCancelled)
+    }
+
+    @Test
+    public fun testCancelling() = runTest {
+        expect(1)
+        val job = launch(NonCancellable, start = CoroutineStart.LAZY) {
+            expect(2)
+            // launches child
+            launch(start = CoroutineStart.ATOMIC) {
+                expect(4)
+            }
+            // completes normally
+        }
+        // New job
+        assertFalse(job.isActive)
+        assertFalse(job.isCompleted)
+        assertFalse(job.isFailed)
+        assertFalse(job.isCancelled)
+        // New -> Active
+        job.start()
+        assertTrue(job.isActive)
+        assertFalse(job.isCompleted)
+        assertFalse(job.isFailed)
+        assertFalse(job.isCancelled)
+        // Active -> Completing
+        yield() // scheduled & starts child
+        expect(3)
+        assertTrue(job.isActive)
+        assertFalse(job.isCompleted)
+        assertFalse(job.isFailed)
+        assertFalse(job.isCancelled)
+        // Completing -> Cancelling
+        job.cancel()
+        assertFalse(job.isActive)
+        assertFalse(job.isCompleted)
+        assertTrue(job.isFailed)
+        assertTrue(job.isCancelled)
+        // Cancelling -> Cancelled
+        yield()
+        finish(5)
+        assertFalse(job.isActive)
+        assertTrue(job.isCompleted)
+        assertTrue(job.isFailed)
+        assertTrue(job.isCancelled)
+    }
+
+    private class TestException : Exception()
+}
\ No newline at end of file
diff --git a/common/kotlinx-coroutines-core-common/test/JobTest.kt b/common/kotlinx-coroutines-core-common/test/JobTest.kt
index 1379861..aaa072c 100644
--- a/common/kotlinx-coroutines-core-common/test/JobTest.kt
+++ b/common/kotlinx-coroutines-core-common/test/JobTest.kt
@@ -164,7 +164,7 @@
             delay(Long.MAX_VALUE)
         }
 
-        job.invokeOnCompletion(onCancelling = true) {
+        job.invokeOnCompletion(onFailing = true) {
             assertNotNull(it)
             expect(3)
         }
diff --git a/common/kotlinx-coroutines-core-common/test/channels/ProduceTest.kt b/common/kotlinx-coroutines-core-common/test/channels/ProduceTest.kt
index 0f958af..cbbda94 100644
--- a/common/kotlinx-coroutines-core-common/test/channels/ProduceTest.kt
+++ b/common/kotlinx-coroutines-core-common/test/channels/ProduceTest.kt
@@ -62,8 +62,7 @@
                 send(2) // will get cancelled
             } catch (e: Exception) {
                 finish(6)
-                check(e is JobCancellationException && e.job == coroutineContext[Job])
-                check(e.cause is TestException)
+                check(e is TestException)
                 throw e
             }
             expectUnreached()
diff --git a/core/kotlinx-coroutines-core/src/Builders.kt b/core/kotlinx-coroutines-core/src/Builders.kt
index 43521c2..e3c2708 100644
--- a/core/kotlinx-coroutines-core/src/Builders.kt
+++ b/core/kotlinx-coroutines-core/src/Builders.kt
@@ -56,7 +56,7 @@
         if (privateEventLoop) require(eventLoop is BlockingEventLoop)
     }
 
-    override fun onCompletionInternal(state: Any?, mode: Int) {
+    override fun onCompletionInternal(state: Any?, mode: Int, suppressed: Boolean) {
         // wake up blocked thread
         if (Thread.currentThread() != blockedThread)
             LockSupport.unpark(blockedThread)
diff --git a/core/kotlinx-coroutines-core/src/channels/Actor.kt b/core/kotlinx-coroutines-core/src/channels/Actor.kt
index dd933e3..882cb5f 100644
--- a/core/kotlinx-coroutines-core/src/channels/Actor.kt
+++ b/core/kotlinx-coroutines-core/src/channels/Actor.kt
@@ -197,9 +197,8 @@
         _channel.cancel(cause)
     }
 
-    override fun handleJobException(exception: Throwable) {
-        handleCoroutineException(context, exception, this)
-    }
+    override fun failParent(exception: Throwable) = failParentImpl(exception)
+    override fun handleJobException(exception: Throwable) = handleExceptionViaHandler(parentContext, exception)
 }
 
 private class LazyActorCoroutine<E>(
diff --git a/core/kotlinx-coroutines-core/test/JobDisposeStressTest.kt b/core/kotlinx-coroutines-core/test/JobDisposeStressTest.kt
index af0a42c..e17c92b 100644
--- a/core/kotlinx-coroutines-core/test/JobDisposeStressTest.kt
+++ b/core/kotlinx-coroutines-core/test/JobDisposeStressTest.kt
@@ -39,7 +39,7 @@
         threads += testThread("creator") {
             while (!done) {
                 val job = TestJob()
-                val handle = job.invokeOnCompletion(onCancelling = true) { /* nothing */ }
+                val handle = job.invokeOnCompletion(onFailing = true) { /* nothing */ }
                 this.job = job // post job to cancelling thread
                 this.handle = handle // post handle to concurrent disposer thread
                 handle.dispose() // dispose of handle from this thread (concurrently with other disposer)
diff --git a/core/kotlinx-coroutines-core/test/JobHandlersUpgradeStressTest.kt b/core/kotlinx-coroutines-core/test/JobHandlersUpgradeStressTest.kt
index 73711f4..dc0a83c 100644
--- a/core/kotlinx-coroutines-core/test/JobHandlersUpgradeStressTest.kt
+++ b/core/kotlinx-coroutines-core/test/JobHandlersUpgradeStressTest.kt
@@ -62,7 +62,7 @@
                     // burn some time
                     repeat(rnd.nextInt(1000)) { sink.incrementAndGet() }
                     val handle =
-                        job.invokeOnCompletion(onCancelling = onCancelling, invokeImmediately = invokeImmediately) {
+                        job.invokeOnCompletion(onFailing = onCancelling, invokeImmediately = invokeImmediately) {
                             if (!state.state.compareAndSet(0, 1))
                                 error("Fired more than once or too late: state=${state.state.value}")
                         }
diff --git a/core/kotlinx-coroutines-core/test/TestBase.kt b/core/kotlinx-coroutines-core/test/TestBase.kt
index 79d66e4..e03242d 100644
--- a/core/kotlinx-coroutines-core/test/TestBase.kt
+++ b/core/kotlinx-coroutines-core/test/TestBase.kt
@@ -59,7 +59,9 @@
         error.compareAndSet(null, cause)
         println("$message: $cause")
         cause.printStackTrace(System.out)
-    } 
+        println("--- Detected at ---")
+        Throwable().printStackTrace(System.out)
+    }
 
     /**
      * Throws [IllegalStateException] when `value` is false like `check` in stdlib, but also ensures that the
diff --git a/core/kotlinx-coroutines-core/test/channels/ActorTest.kt b/core/kotlinx-coroutines-core/test/channels/ActorTest.kt
index 70aa73f..3cfa8ea 100644
--- a/core/kotlinx-coroutines-core/test/channels/ActorTest.kt
+++ b/core/kotlinx-coroutines-core/test/channels/ActorTest.kt
@@ -128,7 +128,7 @@
             job.await()
             expectUnreached()
         } catch (e: JobCancellationException) {
-            assertTrue(e.message?.contains("Job was cancelled normally") ?: false)
+            assertTrue(e.message?.contains("Job was cancelled") ?: false)
         }
 
         finish(3)
diff --git a/core/kotlinx-coroutines-core/test/exceptions/Exceptions.kt b/core/kotlinx-coroutines-core/test/exceptions/Exceptions.kt
index ba8428f..d97a433 100644
--- a/core/kotlinx-coroutines-core/test/exceptions/Exceptions.kt
+++ b/core/kotlinx-coroutines-core/test/exceptions/Exceptions.kt
@@ -34,17 +34,22 @@
 }
 
 class CapturingHandler : AbstractCoroutineContextElement(CoroutineExceptionHandler),
-    CoroutineExceptionHandler {
-    val unhandled: MutableList<Throwable> = Collections.synchronizedList(ArrayList<Throwable>())!!
+    CoroutineExceptionHandler
+{
+    private var unhandled: ArrayList<Throwable>? = ArrayList()
 
-    override fun handleException(context: CoroutineContext, exception: Throwable) {
-        unhandled.add(exception)
+    override fun handleException(context: CoroutineContext, exception: Throwable) = synchronized<Unit>(this) {
+        unhandled!!.add(exception)
     }
 
-    fun getException(): Throwable {
-        val size = unhandled.size
+    fun getExceptions(): List<Throwable> = synchronized(this) {
+        return unhandled!!.also { unhandled = null }
+    }
+
+    fun getException(): Throwable = synchronized(this) {
+        val size = unhandled!!.size
         assert(size == 1) { "Expected one unhandled exception, but have $size: $unhandled" }
-        return unhandled[0]
+        return unhandled!![0].also { unhandled = null }
     }
 }
 
@@ -63,5 +68,5 @@
 ): List<Throwable> {
     val handler = CapturingHandler()
     runBlocking(context + handler, block = block)
-    return handler.unhandled
+    return handler.getExceptions()
 }
diff --git a/core/kotlinx-coroutines-core/test/exceptions/JobExceptionsStressTest.kt b/core/kotlinx-coroutines-core/test/exceptions/JobExceptionsStressTest.kt
index 760cc05..ca44e27 100644
--- a/core/kotlinx-coroutines-core/test/exceptions/JobExceptionsStressTest.kt
+++ b/core/kotlinx-coroutines-core/test/exceptions/JobExceptionsStressTest.kt
@@ -33,36 +33,39 @@
                 val job = launch(NonCancellable) {
                     launch(start = CoroutineStart.ATOMIC) {
                         barrier.await()
-                        throw ArithmeticException()
+                        throw TestException1()
                     }
                     launch(start = CoroutineStart.ATOMIC) {
                         barrier.await()
-                        throw IOException()
+                        throw TestException2()
                     }
                     launch(start = CoroutineStart.ATOMIC) {
                         barrier.await()
-                        throw IllegalArgumentException()
+                        throw TestException3()
                     }
-                    delay(Long.MAX_VALUE)
+                    delay(1000) // to avoid OutOfMemory errors....
                 }
                 barrier.await()
                 job.join()
             }
-
             val classes = mutableSetOf(
-                IllegalArgumentException::class,
-                IOException::class, ArithmeticException::class)
-
+                TestException1::class,
+                TestException2::class,
+                TestException3::class
+            )
             val suppressedExceptions = exception.suppressed().toSet()
             assertTrue(classes.remove(exception::class),
-                "Failed to remove ${exception::class} from $suppressedExceptions")
-
+                "Failed to remove ${exception::class} from $suppressedExceptions"
+            )
             for (throwable in suppressedExceptions.toSet()) { // defensive copy
                 assertTrue(classes.remove(throwable::class),
                     "Failed to remove ${throwable::class} from $suppressedExceptions")
             }
-
             assertTrue(classes.isEmpty(), "Expected all exception to be present, but following exceptions are missing: $classes")
         }
     }
+
+    private class TestException1 : Exception()
+    private class TestException2 : Exception()
+    private class TestException3 : Exception()
 }
diff --git a/core/kotlinx-coroutines-core/test/exceptions/JobNestedExceptionsTest.kt b/core/kotlinx-coroutines-core/test/exceptions/JobNestedExceptionsTest.kt
index 5affae1..5763bb4 100644
--- a/core/kotlinx-coroutines-core/test/exceptions/JobNestedExceptionsTest.kt
+++ b/core/kotlinx-coroutines-core/test/exceptions/JobNestedExceptionsTest.kt
@@ -68,22 +68,19 @@
     fun testNestedAtomicThrow() {
         val exception = runBlock {
             expect(1)
-            val job = launch(NonCancellable, start = CoroutineStart.ATOMIC) {
+            val job = launch(NonCancellable + CoroutineName("outer"), start = CoroutineStart.ATOMIC) {
                 expect(2)
-
-                launch(start = CoroutineStart.ATOMIC) {
-                    expect(3)
+                launch(CoroutineName("nested"), start = CoroutineStart.ATOMIC) {
+                    expect(4)
                     throw IOException()
                 }
-
+                expect(3)
                 throw ArithmeticException()
             }
-
             job.join()
-            finish(4)
+            finish(5)
         }
-
-        assertTrue(exception is ArithmeticException)
+        assertTrue(exception is ArithmeticException, "Found $exception")
         checkException<IOException>(exception.suppressed()[0])
     }
 
@@ -91,31 +88,33 @@
     fun testChildThrowsDuringCompletion() {
         val exceptions = runBlockForMultipleExceptions {
             expect(1)
-            val job = launch(NonCancellable, start = CoroutineStart.ATOMIC) {
+            val job = launch(NonCancellable + CoroutineName("outer"), start = CoroutineStart.ATOMIC) {
                 expect(2)
-                launch(start = CoroutineStart.ATOMIC) {
-                    expect(3)
-                    launch(start = CoroutineStart.ATOMIC) {
+                launch(CoroutineName("nested"), start = CoroutineStart.ATOMIC) {
+                    expect(4)
+                    launch(CoroutineName("nested2"), start = CoroutineStart.ATOMIC) {
                         // This child attaches to the parent and throws after parent completion
-                        expect(4)
+                        expect(6)
                         throw NullPointerException()
                     }
-
+                    expect(5)
                     throw IOException()
                 }
-
+                expect(3)
                 throw ArithmeticException()
             }
 
             job.join()
-            finish(5)
+            finish(7)
         }
 
-        assertEquals(1, exceptions.size)
+        assertEquals(1, exceptions.size, "Found $exceptions")
         val exception = exceptions[0]
+        assertTrue(exception is ArithmeticException, "Exception is $exception")
         val suppressed = exception.suppressed()
-        checkException<IOException>(suppressed[0])
-        checkException<NullPointerException>(suppressed[1])
+        // the order of suppressed exceptions here is a bit hacky, may change in the future
+        checkException<NullPointerException>(suppressed[0])
+        checkException<IOException>(suppressed[1])
         checkCycles(exception)
     }
 }
diff --git a/core/kotlinx-coroutines-core/test/exceptions/SuppresionTests.kt b/core/kotlinx-coroutines-core/test/exceptions/SuppresionTests.kt
index 51498e2..921107d 100644
--- a/core/kotlinx-coroutines-core/test/exceptions/SuppresionTests.kt
+++ b/core/kotlinx-coroutines-core/test/exceptions/SuppresionTests.kt
@@ -5,8 +5,6 @@
 package kotlinx.coroutines.experimental.exceptions
 
 import kotlinx.coroutines.experimental.*
-import kotlinx.coroutines.experimental.exceptions.*
-import kotlinx.coroutines.experimental.selects.*
 import java.io.*
 import kotlin.test.*
 
@@ -59,7 +57,7 @@
             }
         }
 
-        coroutine.invokeOnCompletion(onCancelling = true) {
+        coroutine.invokeOnCompletion(onFailing = true) {
             assertTrue(it is ArithmeticException)
             assertTrue(it!!.suppressed().isEmpty())
             expect(6)