Introduce ThreadContextElement API to integrate with thread-local sensitive code

* Debug thread name is redesigned using ThreadContextElement API
  where the name of thread reflects the name of currently coroutine.
* Intrinsics for startCoroutineUndispatched that correspond to
  CoroutineStart.UNDISPATCHED properly update coroutine context.
* New intrinsics named startCoroutineUnintercepted are introduced.
  They do not update thread context.
* withContext logic is fixed properly update context is various situations.
* DebugThreadNameTest is introduced.
* Reporting of unhandled errors in TestBase is improved.
  Its CoroutineExceptionHandler records but does not rethrow exception.
  This makes sure that failed tests actually fail and do not hang in
  recursive attempt to handle unhandled coroutine exception.

Fixes #119
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 fd84630..6307e06 100644
--- a/binary-compatibility-validator/reference-public-api/kotlinx-coroutines-core.txt
+++ b/binary-compatibility-validator/reference-public-api/kotlinx-coroutines-core.txt
@@ -136,8 +136,6 @@
 	public static final fun newCoroutineContext (Lkotlin/coroutines/experimental/CoroutineContext;)Lkotlin/coroutines/experimental/CoroutineContext;
 	public static final fun newCoroutineContext (Lkotlin/coroutines/experimental/CoroutineContext;Lkotlinx/coroutines/experimental/Job;)Lkotlin/coroutines/experimental/CoroutineContext;
 	public static synthetic fun newCoroutineContext$default (Lkotlin/coroutines/experimental/CoroutineContext;Lkotlinx/coroutines/experimental/Job;ILjava/lang/Object;)Lkotlin/coroutines/experimental/CoroutineContext;
-	public static final fun restoreThreadContext (Ljava/lang/String;)V
-	public static final fun updateThreadContext (Lkotlin/coroutines/experimental/CoroutineContext;)Ljava/lang/String;
 }
 
 public abstract class kotlinx/coroutines/experimental/CoroutineDispatcher : kotlin/coroutines/experimental/AbstractCoroutineContextElement, kotlin/coroutines/experimental/ContinuationInterceptor {
@@ -436,6 +434,18 @@
 	public static synthetic fun withTimeoutOrNull$default (JLjava/util/concurrent/TimeUnit;Lkotlin/jvm/functions/Function2;Lkotlin/coroutines/experimental/Continuation;ILjava/lang/Object;)Ljava/lang/Object;
 }
 
+public abstract interface class kotlinx/coroutines/experimental/ThreadContextElement : kotlin/coroutines/experimental/CoroutineContext$Element {
+	public abstract fun restoreThreadContext (Lkotlin/coroutines/experimental/CoroutineContext;Ljava/lang/Object;)V
+	public abstract fun updateThreadContext (Lkotlin/coroutines/experimental/CoroutineContext;)Ljava/lang/Object;
+}
+
+public final class kotlinx/coroutines/experimental/ThreadContextElement$DefaultImpls {
+	public static fun fold (Lkotlinx/coroutines/experimental/ThreadContextElement;Ljava/lang/Object;Lkotlin/jvm/functions/Function2;)Ljava/lang/Object;
+	public static fun get (Lkotlinx/coroutines/experimental/ThreadContextElement;Lkotlin/coroutines/experimental/CoroutineContext$Key;)Lkotlin/coroutines/experimental/CoroutineContext$Element;
+	public static fun minusKey (Lkotlinx/coroutines/experimental/ThreadContextElement;Lkotlin/coroutines/experimental/CoroutineContext$Key;)Lkotlin/coroutines/experimental/CoroutineContext;
+	public static fun plus (Lkotlinx/coroutines/experimental/ThreadContextElement;Lkotlin/coroutines/experimental/CoroutineContext;)Lkotlin/coroutines/experimental/CoroutineContext;
+}
+
 public final class kotlinx/coroutines/experimental/ThreadPoolDispatcher : kotlinx/coroutines/experimental/ExecutorCoroutineDispatcherBase {
 	public fun close ()V
 	public fun getExecutor ()Ljava/util/concurrent/Executor;
@@ -939,6 +949,8 @@
 public final class kotlinx/coroutines/experimental/intrinsics/UndispatchedKt {
 	public static final fun startCoroutineUndispatched (Lkotlin/jvm/functions/Function1;Lkotlin/coroutines/experimental/Continuation;)V
 	public static final fun startCoroutineUndispatched (Lkotlin/jvm/functions/Function2;Ljava/lang/Object;Lkotlin/coroutines/experimental/Continuation;)V
+	public static final fun startCoroutineUnintercepted (Lkotlin/jvm/functions/Function1;Lkotlin/coroutines/experimental/Continuation;)V
+	public static final fun startCoroutineUnintercepted (Lkotlin/jvm/functions/Function2;Ljava/lang/Object;Lkotlin/coroutines/experimental/Continuation;)V
 	public static final fun startUndispatchedOrReturn (Lkotlinx/coroutines/experimental/AbstractCoroutine;Ljava/lang/Object;Lkotlin/jvm/functions/Function2;)Ljava/lang/Object;
 	public static final fun startUndispatchedOrReturn (Lkotlinx/coroutines/experimental/AbstractCoroutine;Lkotlin/jvm/functions/Function1;)Ljava/lang/Object;
 }
diff --git a/build.gradle b/build.gradle
index e3bd895..187ce66 100644
--- a/build.gradle
+++ b/build.gradle
@@ -110,8 +110,10 @@
     sourceSets {
         main.kotlin.srcDirs = ['src']
         test.kotlin.srcDirs = ['test']
+        // todo: do we still need this workaround?
         if (!projectName.endsWith("-native")) {
             main.resources.srcDirs = ['resources']
+            test.resources.srcDirs = ['test-resources']
         }
     }
 }
diff --git a/common/kotlinx-coroutines-core-common/src/Builders.common.kt b/common/kotlinx-coroutines-core-common/src/Builders.common.kt
index 82e8fea..e5745b9 100644
--- a/common/kotlinx-coroutines-core-common/src/Builders.common.kt
+++ b/common/kotlinx-coroutines-core-common/src/Builders.common.kt
@@ -119,8 +119,11 @@
     // fast path #3 if the new dispatcher is the same as the old one.
     // `equals` is used by design (see equals implementation is wrapper context like ExecutorCoroutineDispatcher)
     if (newContext[ContinuationInterceptor] == oldContext[ContinuationInterceptor]) {
-        val newContinuation = RunContinuationDirect(newContext, uCont)
-        return@sc block.startCoroutineUninterceptedOrReturn(newContinuation)
+        val newContinuation = RunContinuationUnintercepted(newContext, uCont)
+        // There are some other changes in the context, so this thread needs to be updated
+        withCoroutineContext(newContext) {
+            return@sc block.startCoroutineUninterceptedOrReturn(newContinuation)
+        }
     }
     // slowest path otherwise -- use new interceptor, sync to its result via a full-blown instance of RunCompletion
     require(!start.isLazy) { "$start start is not supported" }
@@ -130,7 +133,6 @@
         resumeMode = if (start == CoroutineStart.ATOMIC) MODE_ATOMIC_DEFAULT else MODE_CANCELLABLE
     )
     completion.initParentJobInternal(newContext[Job]) // attach to job
-    @Suppress("DEPRECATION")
     start(block, completion)
     completion.getResult()
 }
@@ -178,10 +180,22 @@
     }
 }
 
-private class RunContinuationDirect<in T>(
+private class RunContinuationUnintercepted<in T>(
     override val context: CoroutineContext,
-    continuation: Continuation<T>
-) : Continuation<T> by continuation
+    private val continuation: Continuation<T>
+): Continuation<T> {
+    override fun resume(value: T) {
+        withCoroutineContext(continuation.context) {
+            continuation.resume(value)
+        }
+    }
+
+    override fun resumeWithException(exception: Throwable) {
+        withCoroutineContext(continuation.context) {
+            continuation.resumeWithException(exception)
+        }
+    }
+}
 
 @Suppress("UNCHECKED_CAST")
 private class RunCompletion<in T>(
diff --git a/common/kotlinx-coroutines-core-common/src/JobSupport.kt b/common/kotlinx-coroutines-core-common/src/JobSupport.kt
index 0f6818e..4ab879a 100644
--- a/common/kotlinx-coroutines-core-common/src/JobSupport.kt
+++ b/common/kotlinx-coroutines-core-common/src/JobSupport.kt
@@ -531,7 +531,7 @@
                 // already complete -- select result
                 if (select.trySelect(null)) {
                     select.completion.context.checkCompletion() // always check for our completion
-                    block.startCoroutineUndispatched(select.completion)
+                    block.startCoroutineUnintercepted(select.completion)
                 }
                 return
             }
@@ -992,7 +992,7 @@
                     if (state is CompletedExceptionally)
                         select.resumeSelectCancellableWithException(state.cause)
                     else
-                        block.startCoroutineUndispatched(state as T, select.completion)
+                        block.startCoroutineUnintercepted(state as T, select.completion)
                 }
                 return
             }
diff --git a/common/kotlinx-coroutines-core-common/src/ResumeMode.kt b/common/kotlinx-coroutines-core-common/src/ResumeMode.kt
index 6faf515..a444347 100644
--- a/common/kotlinx-coroutines-core-common/src/ResumeMode.kt
+++ b/common/kotlinx-coroutines-core-common/src/ResumeMode.kt
@@ -43,7 +43,7 @@
         MODE_ATOMIC_DEFAULT -> intercepted().resume(value)
         MODE_CANCELLABLE -> intercepted().resumeCancellable(value)
         MODE_DIRECT -> resume(value)
-        MODE_UNDISPATCHED -> resume(value)
+        MODE_UNDISPATCHED -> withCoroutineContext(context) { resume(value) }
         MODE_IGNORE -> {}
         else -> error("Invalid mode $mode")
     }
@@ -54,7 +54,7 @@
         MODE_ATOMIC_DEFAULT -> intercepted().resumeWithException(exception)
         MODE_CANCELLABLE -> intercepted().resumeCancellableWithException(exception)
         MODE_DIRECT -> resumeWithException(exception)
-        MODE_UNDISPATCHED -> resumeWithException(exception)
+        MODE_UNDISPATCHED -> withCoroutineContext(context) { resumeWithException(exception) }
         MODE_IGNORE -> {}
         else -> error("Invalid mode $mode")
     }
diff --git a/common/kotlinx-coroutines-core-common/src/channels/AbstractChannel.kt b/common/kotlinx-coroutines-core-common/src/channels/AbstractChannel.kt
index 42fbb20..2a1aa61 100644
--- a/common/kotlinx-coroutines-core-common/src/channels/AbstractChannel.kt
+++ b/common/kotlinx-coroutines-core-common/src/channels/AbstractChannel.kt
@@ -414,7 +414,7 @@
                     offerResult === ALREADY_SELECTED -> return
                     offerResult === OFFER_FAILED -> {} // retry
                     offerResult === OFFER_SUCCESS -> {
-                        block.startCoroutineUndispatched(receiver = this, completion = select.completion)
+                        block.startCoroutineUnintercepted(receiver = this, completion = select.completion)
                         return
                     }
                     offerResult is Closed<*> -> throw offerResult.sendException
@@ -753,7 +753,7 @@
                     pollResult === POLL_FAILED -> {} // retry
                     pollResult is Closed<*> -> throw pollResult.receiveException
                     else -> {
-                        block.startCoroutineUndispatched(pollResult as E, select.completion)
+                        block.startCoroutineUnintercepted(pollResult as E, select.completion)
                         return
                     }
                 }
@@ -788,14 +788,14 @@
                     pollResult is Closed<*> -> {
                         if (pollResult.closeCause == null) {
                             if (select.trySelect(null))
-                                block.startCoroutineUndispatched(null, select.completion)
+                                block.startCoroutineUnintercepted(null, select.completion)
                             return
                         } else
                             throw pollResult.closeCause
                     }
                     else -> {
                         // selected successfully
-                        block.startCoroutineUndispatched(pollResult as E, select.completion)
+                        block.startCoroutineUnintercepted(pollResult as E, select.completion)
                         return
                     }
                 }
diff --git a/common/kotlinx-coroutines-core-common/src/channels/ConflatedBroadcastChannel.kt b/common/kotlinx-coroutines-core-common/src/channels/ConflatedBroadcastChannel.kt
index 34ed4de..3bdd4c4 100644
--- a/common/kotlinx-coroutines-core-common/src/channels/ConflatedBroadcastChannel.kt
+++ b/common/kotlinx-coroutines-core-common/src/channels/ConflatedBroadcastChannel.kt
@@ -262,7 +262,7 @@
             select.resumeSelectCancellableWithException(it.sendException)
             return
         }
-        block.startCoroutineUndispatched(receiver = this, completion = select.completion)
+        block.startCoroutineUnintercepted(receiver = this, completion = select.completion)
     }
 
     @Suppress("DEPRECATION")
diff --git a/common/kotlinx-coroutines-core-common/src/intrinsics/Undispatched.kt b/common/kotlinx-coroutines-core-common/src/intrinsics/Undispatched.kt
index e7ab254..80048a8 100644
--- a/common/kotlinx-coroutines-core-common/src/intrinsics/Undispatched.kt
+++ b/common/kotlinx-coroutines-core-common/src/intrinsics/Undispatched.kt
@@ -9,38 +9,69 @@
 import kotlin.coroutines.experimental.intrinsics.*
 
 /**
- * Use this function to restart coroutine directly from inside of [suspendCoroutine] in the same context.
+ * Use this function to restart coroutine directly from inside of [suspendCoroutine],
+ * when the code is already in the context of this coroutine.
+ * It does not use [ContinuationInterceptor] and does not update context of the current thread.
  */
-@Suppress("PLATFORM_CLASS_MAPPED_TO_KOTLIN", "UNCHECKED_CAST")
-public fun <T> (suspend () -> T).startCoroutineUndispatched(completion: Continuation<T>) {
-    val value = try {
+public fun <T> (suspend () -> T).startCoroutineUnintercepted(completion: Continuation<T>) {
+    startDirect(completion) {
         startCoroutineUninterceptedOrReturn(completion)
-    } catch (e: Throwable) {
-        completion.resumeWithException(e)
-        return
     }
-    if (value !== COROUTINE_SUSPENDED)
-        completion.resume(value as T)
 }
 
 /**
- * Use this function to restart coroutine directly from inside of [suspendCoroutine] in the same context.
+ * Use this function to restart coroutine directly from inside of [suspendCoroutine],
+ * when the code is already in the context of this coroutine.
+ * It does not use [ContinuationInterceptor] and does not update context of the current thread.
  */
-@Suppress("PLATFORM_CLASS_MAPPED_TO_KOTLIN", "UNCHECKED_CAST")
-public fun <R, T> (suspend (R) -> T).startCoroutineUndispatched(receiver: R, completion: Continuation<T>) {
-    val value = try {
+public fun <R, T> (suspend (R) -> T).startCoroutineUnintercepted(receiver: R, completion: Continuation<T>) {
+    startDirect(completion) {
         startCoroutineUninterceptedOrReturn(receiver, completion)
+    }
+}
+
+/**
+ * Use this function to start new coroutine in [CoroutineStart.UNDISPATCHED] mode &mdash;
+ * immediately execute coroutine in the current thread until next suspension.
+ * It does not use [ContinuationInterceptor], but updates the context of the current thread for the new coroutine.
+ */
+public fun <T> (suspend () -> T).startCoroutineUndispatched(completion: Continuation<T>) {
+    startDirect(completion) {
+        withCoroutineContext(completion.context) {
+            startCoroutineUninterceptedOrReturn(completion)
+        }
+    }
+}
+
+/**
+ * Use this function to start new coroutine in [CoroutineStart.UNDISPATCHED] mode &mdash;
+ * immediately execute coroutine in the current thread until next suspension.
+ * It does not use [ContinuationInterceptor], but updates the context of the current thread for the new coroutine.
+ */
+public fun <R, T> (suspend (R) -> T).startCoroutineUndispatched(receiver: R, completion: Continuation<T>) {
+    startDirect(completion) {
+        withCoroutineContext(completion.context) {
+            startCoroutineUninterceptedOrReturn(receiver, completion)
+        }
+    }
+}
+
+private inline fun <T> startDirect(completion: Continuation<T>, block: () -> Any?) {
+    val value = try {
+        block()
     } catch (e: Throwable) {
         completion.resumeWithException(e)
         return
     }
-    if (value !== COROUTINE_SUSPENDED)
+    if (value !== COROUTINE_SUSPENDED) {
+        @Suppress("UNCHECKED_CAST")
         completion.resume(value as T)
+    }
 }
 
 /**
  * Starts this coroutine with the given code [block] in the same context and returns result when it
- * completes without suspnesion.
+ * completes without suspension.
  * This function shall be invoked at most once on this coroutine.
  *
  * First, this function initializes parent job from the `parentContext` of this coroutine that was passed to it
@@ -53,7 +84,7 @@
 
 /**
  * Starts this coroutine with the given code [block] in the same context and returns result when it
- * completes without suspnesion.
+ * completes without suspension.
  * This function shall be invoked at most once on this coroutine.
  *
  * First, this function initializes parent job from the `parentContext` of this coroutine that was passed to it
diff --git a/common/kotlinx-coroutines-core-common/src/selects/Select.kt b/common/kotlinx-coroutines-core-common/src/selects/Select.kt
index eae68ad..23b6752 100644
--- a/common/kotlinx-coroutines-core-common/src/selects/Select.kt
+++ b/common/kotlinx-coroutines-core-common/src/selects/Select.kt
@@ -407,7 +407,7 @@
     override fun onTimeout(time: Long, unit: TimeUnit, block: suspend () -> R) {
         if (time <= 0L) {
             if (trySelect(null))
-                block.startCoroutineUndispatched(completion)
+                block.startCoroutineUnintercepted(completion)
             return
         }
         val action = Runnable {
diff --git a/common/kotlinx-coroutines-core-common/src/sync/Mutex.kt b/common/kotlinx-coroutines-core-common/src/sync/Mutex.kt
index 7657e77..997be13 100644
--- a/common/kotlinx-coroutines-core-common/src/sync/Mutex.kt
+++ b/common/kotlinx-coroutines-core-common/src/sync/Mutex.kt
@@ -252,7 +252,7 @@
                         val failure = select.performAtomicTrySelect(TryLockDesc(this, owner))
                         when {
                             failure == null -> { // success
-                                block.startCoroutineUndispatched(receiver = this, completion = select.completion)
+                                block.startCoroutineUnintercepted(receiver = this, completion = select.completion)
                                 return
                             }
                             failure === ALREADY_SELECTED -> return // already selected -- bail out
diff --git a/common/kotlinx-coroutines-core-common/test/selects/SelectArrayChannelTest.kt b/common/kotlinx-coroutines-core-common/test/selects/SelectArrayChannelTest.kt
index d4a9be4..a805770 100644
--- a/common/kotlinx-coroutines-core-common/test/selects/SelectArrayChannelTest.kt
+++ b/common/kotlinx-coroutines-core-common/test/selects/SelectArrayChannelTest.kt
@@ -289,6 +289,6 @@
     internal fun <R> SelectBuilder<R>.default(block: suspend () -> R) {
         this as SelectBuilderImpl // type assertion
         if (!trySelect(null)) return
-        block.startCoroutineUndispatched(this)
+        block.startCoroutineUnintercepted(this)
     }
 }
diff --git a/common/kotlinx-coroutines-core-common/test/selects/SelectRendezvousChannelTest.kt b/common/kotlinx-coroutines-core-common/test/selects/SelectRendezvousChannelTest.kt
index 6312306..2f7f63b 100644
--- a/common/kotlinx-coroutines-core-common/test/selects/SelectRendezvousChannelTest.kt
+++ b/common/kotlinx-coroutines-core-common/test/selects/SelectRendezvousChannelTest.kt
@@ -310,6 +310,6 @@
     internal fun <R> SelectBuilder<R>.default(block: suspend () -> R) {
         this as SelectBuilderImpl // type assertion
         if (!trySelect(null)) return
-        block.startCoroutineUndispatched(this)
+        block.startCoroutineUnintercepted(this)
     }
 }
diff --git a/core/kotlinx-coroutines-core/src/CoroutineContext.kt b/core/kotlinx-coroutines-core/src/CoroutineContext.kt
index ea8d034..247ba50 100644
--- a/core/kotlinx-coroutines-core/src/CoroutineContext.kt
+++ b/core/kotlinx-coroutines-core/src/CoroutineContext.kt
@@ -4,6 +4,7 @@
 
 package kotlinx.coroutines.experimental
 
+import java.util.*
 import kotlinx.coroutines.experimental.internal.*
 import kotlinx.coroutines.experimental.scheduling.*
 import java.util.concurrent.atomic.*
@@ -98,31 +99,14 @@
  * Executes a block using a given coroutine context.
  */
 internal actual inline fun <T> withCoroutineContext(context: CoroutineContext, block: () -> T): T {
-    val oldName = context.updateThreadContext()
+    val oldValue = updateThreadContext(context)
     try {
         return block()
     } finally {
-        restoreThreadContext(oldName)
+        restoreThreadContext(context, oldValue)
     }
 }
 
-@PublishedApi
-internal fun CoroutineContext.updateThreadContext(): String? {
-    if (!DEBUG) return null
-    val coroutineId = this[CoroutineId] ?: return null
-    val coroutineName = this[CoroutineName]?.name ?: "coroutine"
-    val currentThread = Thread.currentThread()
-    val oldName = currentThread.name
-    currentThread.name = buildString(oldName.length + coroutineName.length + 10) {
-        append(oldName)
-        append(" @")
-        append(coroutineName)
-        append('#')
-        append(coroutineId.id)
-    }
-    return oldName
-}
-
 internal actual val CoroutineContext.coroutineName: String? get() {
     if (!DEBUG) return null
     val coroutineId = this[CoroutineId] ?: return null
@@ -130,12 +114,31 @@
     return "$coroutineName#${coroutineId.id}"
 }
 
-@PublishedApi
-internal fun restoreThreadContext(oldName: String?) {
-    if (oldName != null) Thread.currentThread().name = oldName
-}
+private const val DEBUG_THREAD_NAME_SEPARATOR = " @"
 
-private class CoroutineId(val id: Long) : AbstractCoroutineContextElement(CoroutineId) {
+internal data class CoroutineId(
+    val id: Long
+) : ThreadContextElement<String>, AbstractCoroutineContextElement(CoroutineId) {
     companion object Key : CoroutineContext.Key<CoroutineId>
     override fun toString(): String = "CoroutineId($id)"
+
+    override fun updateThreadContext(context: CoroutineContext): String {
+        val coroutineName = context[CoroutineName]?.name ?: "coroutine"
+        val currentThread = Thread.currentThread()
+        val oldName = currentThread.name
+        var lastIndex = oldName.lastIndexOf(DEBUG_THREAD_NAME_SEPARATOR)
+        if (lastIndex < 0) lastIndex = oldName.length
+        currentThread.name = buildString(lastIndex + coroutineName.length + 10) {
+            append(oldName.substring(0, lastIndex))
+            append(DEBUG_THREAD_NAME_SEPARATOR)
+            append(coroutineName)
+            append('#')
+            append(id)
+        }
+        return oldName
+    }
+
+    override fun restoreThreadContext(context: CoroutineContext, oldState: String) {
+        Thread.currentThread().name = oldState
+    }
 }
diff --git a/core/kotlinx-coroutines-core/src/ThreadContextElement.kt b/core/kotlinx-coroutines-core/src/ThreadContextElement.kt
new file mode 100644
index 0000000..140c9c3
--- /dev/null
+++ b/core/kotlinx-coroutines-core/src/ThreadContextElement.kt
@@ -0,0 +1,153 @@
+/*
+ * Copyright 2016-2018 JetBrains s.r.o. Use of this source code is governed by the Apache 2.0 license.
+ */
+
+package kotlinx.coroutines.experimental
+
+import kotlinx.coroutines.experimental.internal.*
+import kotlin.coroutines.experimental.*
+
+/**
+ * Defines elements in [CoroutineContext] that are installed into thread context
+ * every time the coroutine with this element in the context is resumed on a thread.
+ *
+ * Implementations of this interface define a type [S] of the thread-local state that they need to store on
+ * resume of a coroutine and restore later on suspend and the infrastructure provides the corresponding storage.
+ *
+ * Example usage looks like this:
+ *
+ * ```
+ * // declare thread local variable holding MyData
+ * private val myThreadLocal = ThreadLocal<MyData?>()
+ *
+ * // declare context element holding MyData
+ * class MyElement(val data: MyData) : ThreadContextElement<MyData?> {
+ *     // declare companion object for a key of this element in coroutine context
+ *     companion object Key : CoroutineContext.Key<MyElement>
+ *
+ *     // provide the key of the corresponding context element
+ *     override val key: CoroutineContext.Key<MyElement>
+ *         get() = Key
+ *
+ *     // this is invoked before coroutine is resumed on current thread
+ *     override fun updateThreadContext(context: CoroutineContext): MyData? {
+ *         val oldState = myThreadLocal.get()
+ *         myThreadLocal.set(data)
+ *         return oldState
+ *     }
+ *
+ *     // this is invoked after coroutine has suspended on current thread
+ *     override fun restoreThreadContext(context: CoroutineContext, oldState: MyData?) {
+ *         myThreadLocal.set(oldState)
+ *     }
+ * }
+ * ```
+ */
+public interface ThreadContextElement<S> : CoroutineContext.Element {
+    /**
+     * Updates context of the current thread.
+     * This function is invoked before the coroutine in the specified [context] is resumed in the current thread
+     * when the context of the coroutine this element.
+     * The result of this function is the old value of the thread-local state that will be passed to [restoreThreadContext].
+     *
+     * @param context the coroutine context.
+     */
+    public fun updateThreadContext(context: CoroutineContext): S
+
+    /**
+     * Restores context of the current thread.
+     * This function is invoked after the coroutine in the specified [context] is suspended in the current thread
+     * if [updateThreadContext] was previously invoked on resume of this coroutine.
+     * The value of [oldState] is the result of the previous invocation of [updateThreadContext] and it should
+     * be restored in the thread-local state by this function.
+     *
+     * @param context the coroutine context.
+     * @param oldState the value returned by the previous invocation of [updateThreadContext].
+     */
+    public fun restoreThreadContext(context: CoroutineContext, oldState: S)
+}
+
+private val ZERO = Symbol("ZERO")
+
+// Used when there are >= 2 active elements in the context
+private class ThreadState(val context: CoroutineContext, n: Int) {
+    private var a = arrayOfNulls<Any>(n)
+    private var i = 0
+
+    fun append(value: Any?) { a[i++] = value }
+    fun take() = a[i++]
+    fun start() { i = 0 }
+}
+
+// Counts ThreadContextElements in the context
+// Any? here is Int | ThreadContextElement (when count is one)
+private val countAll =
+    fun (countOrElement: Any?, element: CoroutineContext.Element): Any? {
+        if (element is ThreadContextElement<*>) {
+            val inCount = countOrElement as? Int ?: 1
+            return if (inCount == 0) element else inCount + 1
+        }
+        return countOrElement
+    }
+
+// Find one (first) ThreadContextElement in the context, it is used when we know there is exactly one
+private val findOne =
+    fun (found: ThreadContextElement<*>?, element: CoroutineContext.Element): ThreadContextElement<*>? {
+        if (found != null) return found
+        return element as? ThreadContextElement<*>
+    }
+
+// Updates state for ThreadContextElements in the context using the given ThreadState
+private val updateState =
+    fun (state: ThreadState, element: CoroutineContext.Element): ThreadState {
+        if (element is ThreadContextElement<*>) {
+            state.append(element.updateThreadContext(state.context))
+        }
+        return state
+    }
+
+// Restores state for all ThreadContextElements in the context from the given ThreadState
+private val restoreState =
+    fun (state: ThreadState, element: CoroutineContext.Element): ThreadState {
+        @Suppress("UNCHECKED_CAST")
+        if (element is ThreadContextElement<*>) {
+            (element as ThreadContextElement<Any?>).restoreThreadContext(state.context, state.take())
+        }
+        return state
+    }
+
+internal fun updateThreadContext(context: CoroutineContext): Any? {
+    val count = context.fold(0, countAll)
+    @Suppress("IMPLICIT_BOXING_IN_IDENTITY_EQUALS")
+    return when {
+        count === 0 -> ZERO // very fast path when there are no active ThreadContextElements
+        //    ^^^ identity comparison for speed, we know zero always has the same identity
+        count is Int -> {
+            // slow path for multiple active ThreadContextElements, allocates ThreadState for multiple old values
+            context.fold(ThreadState(context, count), updateState)
+        }
+        else -> {
+            // fast path for one ThreadContextElement (no allocations, no additional context scan)
+            @Suppress("UNCHECKED_CAST")
+            val element = count as ThreadContextElement<Any?>
+            element.updateThreadContext(context)
+        }
+    }
+}
+
+internal fun restoreThreadContext(context: CoroutineContext, oldState: Any?) {
+    when {
+        oldState === ZERO -> return // very fast path when there are no ThreadContextElements
+        oldState is ThreadState -> {
+            // slow path with multiple stored ThreadContextElements
+            oldState.start()
+            context.fold(oldState, restoreState)
+        }
+        else -> {
+            // fast path for one ThreadContextElement, but need to find it
+            @Suppress("UNCHECKED_CAST")
+            val element = context.fold(null, findOne) as ThreadContextElement<Any?>
+            element.restoreThreadContext(context, oldState)
+        }
+    }
+}
diff --git a/core/kotlinx-coroutines-core/test/DebugThreadNameTest.kt b/core/kotlinx-coroutines-core/test/DebugThreadNameTest.kt
new file mode 100644
index 0000000..ff91555
--- /dev/null
+++ b/core/kotlinx-coroutines-core/test/DebugThreadNameTest.kt
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2016-2018 JetBrains s.r.o. Use of this source code is governed by the Apache 2.0 license.
+ */
+
+package kotlinx.coroutines.experimental
+
+import kotlin.coroutines.experimental.*
+import kotlin.test.*
+
+class DebugThreadNameTest : TestBase() {
+    @BeforeTest
+    fun resetName() {
+        resetCoroutineId()
+    }
+
+    @Test
+    fun testLaunchId() = runTest {
+        assertName("coroutine#1")
+        launch(coroutineContext) {
+            assertName("coroutine#2")
+            yield()
+            assertName("coroutine#2")
+        }
+        assertName("coroutine#1")
+    }
+
+    @Test
+    fun testLaunchIdUndispatched() = runTest {
+        assertName("coroutine#1")
+        launch(coroutineContext, start = CoroutineStart.UNDISPATCHED) {
+            assertName("coroutine#2")
+            yield()
+            assertName("coroutine#2")
+        }
+        assertName("coroutine#1")
+    }
+
+    @Test
+    fun testLaunchName() = runTest {
+        assertName("coroutine#1")
+        launch(coroutineContext + CoroutineName("TEST")) {
+            assertName("TEST#2")
+            yield()
+            assertName("TEST#2")
+        }
+        assertName("coroutine#1")
+    }
+
+    @Test
+    fun testWithContext() = runTest {
+        assertName("coroutine#1")
+        withContext(DefaultDispatcher) {
+            assertName("coroutine#1")
+            yield()
+            assertName("coroutine#1")
+            withContext(CoroutineName("TEST")) {
+                assertName("TEST#1")
+                yield()
+                assertName("TEST#1")
+            }
+            assertName("coroutine#1")
+            yield()
+            assertName("coroutine#1")
+        }
+        assertName("coroutine#1")
+    }
+
+    private fun assertName(expected: String) {
+        val name = Thread.currentThread().name
+        val split = name.split(Regex(" @"))
+        assertEquals(2, split.size, "Thread name '$name' is expected to contain one coroutine name")
+        assertEquals(expected, split[1], "Thread name '$name' is expected to end with coroutine name '$expected'")
+    }
+}
\ No newline at end of file
diff --git a/core/kotlinx-coroutines-core/test/TestBase.kt b/core/kotlinx-coroutines-core/test/TestBase.kt
index 2ef6cdd..aa5a1e6 100644
--- a/core/kotlinx-coroutines-core/test/TestBase.kt
+++ b/core/kotlinx-coroutines-core/test/TestBase.kt
@@ -55,6 +55,12 @@
         throw exception
     }
 
+    private fun printError(message: String, cause: Throwable) {
+        error.compareAndSet(null, cause)
+        println("$message: $cause")
+        cause.printStackTrace(System.out)
+    } 
+
     /**
      * Throws [IllegalStateException] when `value` is false like `check` in stdlib, but also ensures that the
      * test will not complete successfully even if this exception is consumed somewhere in the test.
@@ -132,10 +138,12 @@
             runBlocking(block = block, context = CoroutineExceptionHandler { context, e ->
                 if (e is CancellationException) return@CoroutineExceptionHandler // are ignored
                 exCount++
-                if (exCount > unhandled.size)
-                    error("Too many unhandled exceptions $exCount, expected ${unhandled.size}, got: $e", e)
-                if (!unhandled[exCount - 1](e))
-                    error("Unhandled exception was unexpected: $e", e)
+                when {
+                    exCount > unhandled.size ->
+                        printError("Too many unhandled exceptions $exCount, expected ${unhandled.size}, got: $e", e)
+                    !unhandled[exCount - 1](e) ->
+                        printError("Unhandled exception was unexpected: $e", e)
+                }
                 context[Job]?.cancel(e)
             })
         } catch (e: Throwable) {
diff --git a/core/kotlinx-coroutines-core/test/ThreadContextElementTest.kt b/core/kotlinx-coroutines-core/test/ThreadContextElementTest.kt
new file mode 100644
index 0000000..0c670f2
--- /dev/null
+++ b/core/kotlinx-coroutines-core/test/ThreadContextElementTest.kt
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2016-2018 JetBrains s.r.o. Use of this source code is governed by the Apache 2.0 license.
+ */
+
+package kotlinx.coroutines.experimental
+
+import org.junit.Test
+import kotlin.coroutines.experimental.*
+import kotlin.test.*
+
+class ThreadContextElementTest : TestBase() {
+    @Test
+    fun testExample() = runTest {
+        val exceptionHandler = coroutineContext[CoroutineExceptionHandler]!!
+        val mainDispatcher = coroutineContext[ContinuationInterceptor]!!
+        val mainThread = Thread.currentThread()
+        val data = MyData()
+        val element = MyElement(data)
+        assertNull(myThreadLocal.get())
+        val job = launch(element + exceptionHandler) {
+            assertTrue(mainThread != Thread.currentThread())
+            assertSame(element, coroutineContext[MyElement])
+            assertSame(data, myThreadLocal.get())
+            withContext(mainDispatcher) {
+                assertSame(mainThread, Thread.currentThread())
+                assertSame(element, coroutineContext[MyElement])
+                assertSame(data, myThreadLocal.get())
+            }
+            assertTrue(mainThread != Thread.currentThread())
+            assertSame(element, coroutineContext[MyElement])
+            assertSame(data, myThreadLocal.get())
+        }
+        assertNull(myThreadLocal.get())
+        job.join()
+        assertNull(myThreadLocal.get())
+    }
+
+    @Test
+    fun testUndispatched()= runTest {
+        val exceptionHandler = coroutineContext[CoroutineExceptionHandler]!!
+        val data = MyData()
+        val element = MyElement(data)
+        val job = launch(
+            context = DefaultDispatcher + exceptionHandler + element,
+            start = CoroutineStart.UNDISPATCHED
+        ) {
+            assertSame(data, myThreadLocal.get())
+            yield()
+            assertSame(data, myThreadLocal.get())
+        }
+        assertNull(myThreadLocal.get())
+        job.join()
+        assertNull(myThreadLocal.get())
+    }
+}
+
+class MyData
+
+// declare thread local variable holding MyData
+private val myThreadLocal = ThreadLocal<MyData?>()
+
+// declare context element holding MyData
+class MyElement(val data: MyData) : ThreadContextElement<MyData?> {
+    // declare companion object for a key of this element in coroutine context
+    companion object Key : CoroutineContext.Key<MyElement>
+
+    // provide the key of the corresponding context element
+    override val key: CoroutineContext.Key<MyElement>
+        get() = Key
+
+    // this is invoked before coroutine is resumed on current thread
+    override fun updateThreadContext(context: CoroutineContext): MyData? {
+        val oldState = myThreadLocal.get()
+        myThreadLocal.set(data)
+        return oldState
+    }
+
+    // this is invoked after coroutine has suspended on current thread
+    override fun restoreThreadContext(context: CoroutineContext, oldState: MyData?) {
+        myThreadLocal.set(oldState)
+    }
+}
diff --git a/core/kotlinx-coroutines-core/test/selects/SelectChannelStressTest.kt b/core/kotlinx-coroutines-core/test/selects/SelectChannelStressTest.kt
index 95d661c..a83b1c4 100644
--- a/core/kotlinx-coroutines-core/test/selects/SelectChannelStressTest.kt
+++ b/core/kotlinx-coroutines-core/test/selects/SelectChannelStressTest.kt
@@ -71,6 +71,6 @@
     internal fun <R> SelectBuilder<R>.default(block: suspend () -> R) {
         this as SelectBuilderImpl // type assertion
         if (!trySelect(null)) return
-        block.startCoroutineUndispatched(this)
+        block.startCoroutineUnintercepted(this)
     }
 }
diff --git a/integration/kotlinx-coroutines-quasar/src/Quasar.kt b/integration/kotlinx-coroutines-quasar/src/Quasar.kt
index 73a7f71..7dd693e 100644
--- a/integration/kotlinx-coroutines-quasar/src/Quasar.kt
+++ b/integration/kotlinx-coroutines-quasar/src/Quasar.kt
@@ -43,7 +43,8 @@
 private class CoroutineAsync<T>(
     private val block: suspend () -> T
 ) : FiberAsync<T, Throwable>(), Continuation<T> {
-    override val context: CoroutineContext = Fiber.currentFiber().scheduler.executor.asCoroutineDispatcher()
+    override val context: CoroutineContext =
+        newCoroutineContext(Fiber.currentFiber().scheduler.executor.asCoroutineDispatcher())
     override fun resume(value: T) { asyncCompleted(value) }
     override fun resumeWithException(exception: Throwable) { asyncFailed(exception) }
 
diff --git a/js/kotlinx-coroutines-core-js/test/TestBase.kt b/js/kotlinx-coroutines-core-js/test/TestBase.kt
index 061b1d7..8db40db 100644
--- a/js/kotlinx-coroutines-core-js/test/TestBase.kt
+++ b/js/kotlinx-coroutines-core-js/test/TestBase.kt
@@ -27,6 +27,12 @@
         throw exception
     }
 
+    private fun printError(message: String, cause: Throwable) {
+        if (error == null) error = cause
+        println("$message: $cause")
+        console.log(cause)
+    }
+
     /**
      * Asserts that this invocation is `index`-th in the execution sequence (counting from one).
      */
@@ -69,10 +75,12 @@
         return promise(block = block, context = CoroutineExceptionHandler { context, e ->
             if (e is CancellationException) return@CoroutineExceptionHandler // are ignored
             exCount++
-            if (exCount > unhandled.size)
-                error("Too many unhandled exceptions $exCount, expected ${unhandled.size}", e)
-            if (!unhandled[exCount - 1](e))
-                error("Unhandled exception was unexpected", e)
+            when {
+                exCount > unhandled.size ->
+                    printError("Too many unhandled exceptions $exCount, expected ${unhandled.size}, got: $e", e)
+                !unhandled[exCount - 1](e) ->
+                    printError("Unhandled exception was unexpected: $e", e)
+            }
             context[Job]?.cancel(e)
         }).catch { e ->
             ex = e
diff --git a/native/kotlinx-coroutines-core-native/test/TestBase.kt b/native/kotlinx-coroutines-core-native/test/TestBase.kt
index f287352..7f75e44 100644
--- a/native/kotlinx-coroutines-core-native/test/TestBase.kt
+++ b/native/kotlinx-coroutines-core-native/test/TestBase.kt
@@ -23,6 +23,11 @@
         throw exception
     }
 
+    private fun printError(message: String, cause: Throwable) {
+        if (error == null) error = cause
+        println("$message: $cause")
+    }
+
     /**
      * Asserts that this invocation is `index`-th in the execution sequence (counting from one).
      */
@@ -65,10 +70,12 @@
             runBlocking(block = block, context = CoroutineExceptionHandler { context, e ->
                 if (e is CancellationException) return@CoroutineExceptionHandler // are ignored
                 exCount++
-                if (exCount > unhandled.size)
-                    error("Too many unhandled exceptions $exCount, expected ${unhandled.size}, got: $e", e)
-                if (!unhandled[exCount - 1](e))
-                    error("Unhandled exception was unexpected: $e", e)
+                when {
+                    exCount > unhandled.size ->
+                        printError("Too many unhandled exceptions $exCount, expected ${unhandled.size}, got: $e", e)
+                    !unhandled[exCount - 1](e) ->
+                        printError("Unhandled exception was unexpected: $e", e)
+                }
                 context[Job]?.cancel(e)
             })
         } catch (e: Throwable) {