aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/org/apache/spark/util/UninterruptibleThread.scala
blob: 4dcf95177aa78b56ff1f8bd1d400b2e94f18fac2 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.spark.util

import javax.annotation.concurrent.GuardedBy

/**
 * A special Thread that provides "runUninterruptibly" to allow running codes without being
 * interrupted by `Thread.interrupt()`. If `Thread.interrupt()` is called during runUninterruptibly
 * is running, it won't set the interrupted status. Instead, setting the interrupted status will be
 * deferred until it's returning from "runUninterruptibly".
 *
 * Note: "runUninterruptibly" should be called only in `this` thread.
 */
private[spark] class UninterruptibleThread(name: String) extends Thread(name) {

  /** A monitor to protect "uninterruptible" and "interrupted" */
  private val uninterruptibleLock = new Object

  /**
   * Indicates if `this`  thread are in the uninterruptible status. If so, interrupting
   * "this" will be deferred until `this`  enters into the interruptible status.
   */
  @GuardedBy("uninterruptibleLock")
  private var uninterruptible = false

  /**
   * Indicates if we should interrupt `this` when we are leaving the uninterruptible zone.
   */
  @GuardedBy("uninterruptibleLock")
  private var shouldInterruptThread = false

  /**
   * Run `f` uninterruptibly in `this` thread. The thread won't be interrupted before returning
   * from `f`.
   *
   * If this method finds that `interrupt` is called before calling `f` and it's not inside another
   * `runUninterruptibly`, it will throw `InterruptedException`.
   *
   * Note: this method should be called only in `this` thread.
   */
  def runUninterruptibly[T](f: => T): T = {
    if (Thread.currentThread() != this) {
      throw new IllegalStateException(s"Call runUninterruptibly in a wrong thread. " +
        s"Expected: $this but was ${Thread.currentThread()}")
    }

    if (uninterruptibleLock.synchronized { uninterruptible }) {
      // We are already in the uninterruptible status. So just run "f" and return
      return f
    }

    uninterruptibleLock.synchronized {
      // Clear the interrupted status if it's set.
      if (Thread.interrupted() || shouldInterruptThread) {
        shouldInterruptThread = false
        // Since it's interrupted, we don't need to run `f` which may be a long computation.
        // Throw InterruptedException as we don't have a T to return.
        throw new InterruptedException()
      }
      uninterruptible = true
    }
    try {
      f
    } finally {
      uninterruptibleLock.synchronized {
        uninterruptible = false
        if (shouldInterruptThread) {
          // Recover the interrupted status
          super.interrupt()
          shouldInterruptThread = false
        }
      }
    }
  }

  /**
   * Tests whether `interrupt()` has been called.
   */
  override def isInterrupted: Boolean = {
    super.isInterrupted || uninterruptibleLock.synchronized { shouldInterruptThread }
  }

  /**
   * Interrupt `this` thread if possible. If `this` is in the uninterruptible status, it won't be
   * interrupted until it enters into the interruptible status.
   */
  override def interrupt(): Unit = {
    uninterruptibleLock.synchronized {
      if (uninterruptible) {
        shouldInterruptThread = true
      } else {
        super.interrupt()
      }
    }
  }
}