Skip to content

Commit f261797

Browse files
author
Andrew Or
committed
Merge branch 'master' of github.com:apache/spark into dag-viz-skipped
2 parents 0eda358 + 775e6f9 commit f261797

File tree

48 files changed

+1338
-952
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

48 files changed

+1338
-952
lines changed

core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,7 @@ import org.apache.spark.deploy.worker.WorkerWatcher
3333
import org.apache.spark.scheduler.TaskDescription
3434
import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
3535
import org.apache.spark.serializer.SerializerInstance
36-
import org.apache.spark.util.{SignalLogger, Utils}
36+
import org.apache.spark.util.{ThreadUtils, SignalLogger, Utils}
3737

3838
private[spark] class CoarseGrainedExecutorBackend(
3939
override val rpcEnv: RpcEnv,
@@ -55,18 +55,19 @@ private[spark] class CoarseGrainedExecutorBackend(
5555
private[this] val ser: SerializerInstance = env.closureSerializer.newInstance()
5656

5757
override def onStart() {
58-
import scala.concurrent.ExecutionContext.Implicits.global
5958
logInfo("Connecting to driver: " + driverUrl)
6059
rpcEnv.asyncSetupEndpointRefByURI(driverUrl).flatMap { ref =>
60+
// This is a very fast action so we can use "ThreadUtils.sameThread"
6161
driver = Some(ref)
6262
ref.ask[RegisteredExecutor.type](
6363
RegisterExecutor(executorId, self, hostPort, cores, extractLogUrls))
64-
} onComplete {
64+
}(ThreadUtils.sameThread).onComplete {
65+
// This is a very fast action so we can use "ThreadUtils.sameThread"
6566
case Success(msg) => Utils.tryLogNonFatalError {
6667
Option(self).foreach(_.send(msg)) // msg must be RegisteredExecutor
6768
}
6869
case Failure(e) => logError(s"Cannot register with driver: $driverUrl", e)
69-
}
70+
}(ThreadUtils.sameThread)
7071
}
7172

7273
def extractLogUrls: Map[String, String] = {

core/src/main/scala/org/apache/spark/io/CompressionCodec.scala

Lines changed: 47 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.spark.io
1919

20-
import java.io.{InputStream, OutputStream}
20+
import java.io.{IOException, InputStream, OutputStream}
2121

2222
import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream}
2323
import net.jpountz.lz4.{LZ4BlockInputStream, LZ4BlockOutputStream}
@@ -154,8 +154,53 @@ class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec {
154154

155155
override def compressedOutputStream(s: OutputStream): OutputStream = {
156156
val blockSize = conf.getSizeAsBytes("spark.io.compression.snappy.blockSize", "32k").toInt
157-
new SnappyOutputStream(s, blockSize)
157+
new SnappyOutputStreamWrapper(new SnappyOutputStream(s, blockSize))
158158
}
159159

160160
override def compressedInputStream(s: InputStream): InputStream = new SnappyInputStream(s)
161161
}
162+
163+
/**
164+
* Wrapper over [[SnappyOutputStream]] which guards against write-after-close and double-close
165+
* issues. See SPARK-7660 for more details. This wrapping can be removed if we upgrade to a version
166+
* of snappy-java that contains the fix for https://github.com/xerial/snappy-java/issues/107.
167+
*/
168+
private final class SnappyOutputStreamWrapper(os: SnappyOutputStream) extends OutputStream {
169+
170+
private[this] var closed: Boolean = false
171+
172+
override def write(b: Int): Unit = {
173+
if (closed) {
174+
throw new IOException("Stream is closed")
175+
}
176+
os.write(b)
177+
}
178+
179+
override def write(b: Array[Byte]): Unit = {
180+
if (closed) {
181+
throw new IOException("Stream is closed")
182+
}
183+
os.write(b)
184+
}
185+
186+
override def write(b: Array[Byte], off: Int, len: Int): Unit = {
187+
if (closed) {
188+
throw new IOException("Stream is closed")
189+
}
190+
os.write(b, off, len)
191+
}
192+
193+
override def flush(): Unit = {
194+
if (closed) {
195+
throw new IOException("Stream is closed")
196+
}
197+
os.flush()
198+
}
199+
200+
override def close(): Unit = {
201+
if (!closed) {
202+
closed = true
203+
os.close()
204+
}
205+
}
206+
}

core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala

Lines changed: 11 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,8 +19,10 @@ package org.apache.spark.rdd
1919

2020
import java.util.concurrent.atomic.AtomicLong
2121

22+
import org.apache.spark.util.ThreadUtils
23+
2224
import scala.collection.mutable.ArrayBuffer
23-
import scala.concurrent.ExecutionContext.Implicits.global
25+
import scala.concurrent.ExecutionContext
2426
import scala.reflect.ClassTag
2527

2628
import org.apache.spark.{ComplexFutureAction, FutureAction, Logging}
@@ -66,6 +68,8 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi
6668
val f = new ComplexFutureAction[Seq[T]]
6769

6870
f.run {
71+
// This is a blocking action so we should use "AsyncRDDActions.futureExecutionContext" which
72+
// is a cached thread pool.
6973
val results = new ArrayBuffer[T](num)
7074
val totalParts = self.partitions.length
7175
var partsScanned = 0
@@ -101,7 +105,7 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi
101105
partsScanned += numPartsToTry
102106
}
103107
results.toSeq
104-
}
108+
}(AsyncRDDActions.futureExecutionContext)
105109

106110
f
107111
}
@@ -123,3 +127,8 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi
123127
(index, data) => Unit, Unit)
124128
}
125129
}
130+
131+
private object AsyncRDDActions {
132+
val futureExecutionContext = ExecutionContext.fromExecutorService(
133+
ThreadUtils.newDaemonCachedThreadPool("AsyncRDDActions-future", 128))
134+
}

core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala

Lines changed: 21 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,9 @@ class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long
4040
var metrics: TaskMetrics)
4141
extends TaskResult[T] with Externalizable {
4242

43+
private var valueObjectDeserialized = false
44+
private var valueObject: T = _
45+
4346
def this() = this(null.asInstanceOf[ByteBuffer], null, null)
4447

4548
override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException {
@@ -72,10 +75,26 @@ class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long
7275
}
7376
}
7477
metrics = in.readObject().asInstanceOf[TaskMetrics]
78+
valueObjectDeserialized = false
7579
}
7680

81+
/**
82+
* When `value()` is called at the first time, it needs to deserialize `valueObject` from
83+
* `valueBytes`. It may cost dozens of seconds for a large instance. So when calling `value` at
84+
* the first time, the caller should avoid to block other threads.
85+
*
86+
* After the first time, `value()` is trivial and just returns the deserialized `valueObject`.
87+
*/
7788
def value(): T = {
78-
val resultSer = SparkEnv.get.serializer.newInstance()
79-
resultSer.deserialize(valueBytes)
89+
if (valueObjectDeserialized) {
90+
valueObject
91+
} else {
92+
// This should not run when holding a lock because it may cost dozens of seconds for a large
93+
// value.
94+
val resultSer = SparkEnv.get.serializer.newInstance()
95+
valueObject = resultSer.deserialize(valueBytes)
96+
valueObjectDeserialized = true
97+
valueObject
98+
}
8099
}
81100
}

core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -54,6 +54,10 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul
5454
if (!taskSetManager.canFetchMoreResults(serializedData.limit())) {
5555
return
5656
}
57+
// deserialize "value" without holding any lock so that it won't block other threads.
58+
// We should call it here, so that when it's called again in
59+
// "TaskSetManager.handleSuccessfulTask", it does not need to deserialize the value.
60+
directResult.value()
5761
(directResult, serializedData.limit())
5862
case IndirectTaskResult(blockId, size) =>
5963
if (!taskSetManager.canFetchMoreResults(size)) {

core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -620,6 +620,12 @@ private[spark] class TaskSetManager(
620620
val index = info.index
621621
info.markSuccessful()
622622
removeRunningTask(tid)
623+
// This method is called by "TaskSchedulerImpl.handleSuccessfulTask" which holds the
624+
// "TaskSchedulerImpl" lock until exiting. To avoid the SPARK-7655 issue, we should not
625+
// "deserialize" the value when holding a lock to avoid blocking other threads. So we call
626+
// "result.value()" in "TaskResultGetter.enqueueSuccessfulTask" before reaching here.
627+
// Note: "result.value()" only deserializes the value when it's called at the first time, so
628+
// here "result.value()" just returns the value and won't block other threads.
623629
sched.dagScheduler.taskEnded(
624630
tasks(index), Success, result.value(), result.accumUpdates, info, result.metrics)
625631
if (!successful(index)) {

core/src/main/scala/org/apache/spark/storage/BlockManager.scala

Lines changed: 13 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -21,8 +21,7 @@ import java.io.{BufferedOutputStream, ByteArrayOutputStream, File, InputStream,
2121
import java.nio.{ByteBuffer, MappedByteBuffer}
2222

2323
import scala.collection.mutable.{ArrayBuffer, HashMap}
24-
import scala.concurrent.{Await, Future}
25-
import scala.concurrent.ExecutionContext.Implicits.global
24+
import scala.concurrent.{ExecutionContext, Await, Future}
2625
import scala.concurrent.duration._
2726
import scala.util.Random
2827

@@ -77,6 +76,9 @@ private[spark] class BlockManager(
7776

7877
private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo]
7978

79+
private val futureExecutionContext = ExecutionContext.fromExecutorService(
80+
ThreadUtils.newDaemonCachedThreadPool("block-manager-future", 128))
81+
8082
// Actual storage of where blocks are kept
8183
private var externalBlockStoreInitialized = false
8284
private[spark] val memoryStore = new MemoryStore(this, maxMemory)
@@ -266,11 +268,13 @@ private[spark] class BlockManager(
266268
asyncReregisterLock.synchronized {
267269
if (asyncReregisterTask == null) {
268270
asyncReregisterTask = Future[Unit] {
271+
// This is a blocking action and should run in futureExecutionContext which is a cached
272+
// thread pool
269273
reregister()
270274
asyncReregisterLock.synchronized {
271275
asyncReregisterTask = null
272276
}
273-
}
277+
}(futureExecutionContext)
274278
}
275279
}
276280
}
@@ -744,7 +748,11 @@ private[spark] class BlockManager(
744748
case b: ByteBufferValues if putLevel.replication > 1 =>
745749
// Duplicate doesn't copy the bytes, but just creates a wrapper
746750
val bufferView = b.buffer.duplicate()
747-
Future { replicate(blockId, bufferView, putLevel) }
751+
Future {
752+
// This is a blocking action and should run in futureExecutionContext which is a cached
753+
// thread pool
754+
replicate(blockId, bufferView, putLevel)
755+
}(futureExecutionContext)
748756
case _ => null
749757
}
750758

@@ -1218,6 +1226,7 @@ private[spark] class BlockManager(
12181226
}
12191227
metadataCleaner.cancel()
12201228
broadcastCleaner.cancel()
1229+
futureExecutionContext.shutdownNow()
12211230
logInfo("BlockManager stopped")
12221231
}
12231232
}

core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala

Lines changed: 18 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -17,13 +17,14 @@
1717

1818
package org.apache.spark.storage
1919

20+
import scala.collection.Iterable
21+
import scala.collection.generic.CanBuildFrom
2022
import scala.concurrent.{Await, Future}
21-
import scala.concurrent.ExecutionContext.Implicits.global
2223

2324
import org.apache.spark.rpc.RpcEndpointRef
2425
import org.apache.spark.{Logging, SparkConf, SparkException}
2526
import org.apache.spark.storage.BlockManagerMessages._
26-
import org.apache.spark.util.RpcUtils
27+
import org.apache.spark.util.{ThreadUtils, RpcUtils}
2728

2829
private[spark]
2930
class BlockManagerMaster(
@@ -102,8 +103,8 @@ class BlockManagerMaster(
102103
val future = driverEndpoint.askWithRetry[Future[Seq[Int]]](RemoveRdd(rddId))
103104
future.onFailure {
104105
case e: Exception =>
105-
logWarning(s"Failed to remove RDD $rddId - ${e.getMessage}}")
106-
}
106+
logWarning(s"Failed to remove RDD $rddId - ${e.getMessage}}", e)
107+
}(ThreadUtils.sameThread)
107108
if (blocking) {
108109
Await.result(future, timeout)
109110
}
@@ -114,8 +115,8 @@ class BlockManagerMaster(
114115
val future = driverEndpoint.askWithRetry[Future[Seq[Boolean]]](RemoveShuffle(shuffleId))
115116
future.onFailure {
116117
case e: Exception =>
117-
logWarning(s"Failed to remove shuffle $shuffleId - ${e.getMessage}}")
118-
}
118+
logWarning(s"Failed to remove shuffle $shuffleId - ${e.getMessage}}", e)
119+
}(ThreadUtils.sameThread)
119120
if (blocking) {
120121
Await.result(future, timeout)
121122
}
@@ -128,8 +129,8 @@ class BlockManagerMaster(
128129
future.onFailure {
129130
case e: Exception =>
130131
logWarning(s"Failed to remove broadcast $broadcastId" +
131-
s" with removeFromMaster = $removeFromMaster - ${e.getMessage}}")
132-
}
132+
s" with removeFromMaster = $removeFromMaster - ${e.getMessage}}", e)
133+
}(ThreadUtils.sameThread)
133134
if (blocking) {
134135
Await.result(future, timeout)
135136
}
@@ -169,11 +170,17 @@ class BlockManagerMaster(
169170
val response = driverEndpoint.
170171
askWithRetry[Map[BlockManagerId, Future[Option[BlockStatus]]]](msg)
171172
val (blockManagerIds, futures) = response.unzip
172-
val result = Await.result(Future.sequence(futures), timeout)
173-
if (result == null) {
173+
implicit val sameThread = ThreadUtils.sameThread
174+
val cbf =
175+
implicitly[
176+
CanBuildFrom[Iterable[Future[Option[BlockStatus]]],
177+
Option[BlockStatus],
178+
Iterable[Option[BlockStatus]]]]
179+
val blockStatus = Await.result(
180+
Future.sequence[Option[BlockStatus], Iterable](futures)(cbf, ThreadUtils.sameThread), timeout)
181+
if (blockStatus == null) {
174182
throw new SparkException("BlockManager returned null for BlockStatus query: " + blockId)
175183
}
176-
val blockStatus = result.asInstanceOf[Iterable[Option[BlockStatus]]]
177184
blockManagerIds.zip(blockStatus).flatMap { case (blockManagerId, status) =>
178185
status.map { s => (blockManagerId, s) }
179186
}.toMap

core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java

Lines changed: 0 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,6 @@
3535
import org.mockito.MockitoAnnotations;
3636
import org.mockito.invocation.InvocationOnMock;
3737
import org.mockito.stubbing.Answer;
38-
import org.xerial.snappy.buffer.CachedBufferAllocator;
3938
import static org.hamcrest.MatcherAssert.assertThat;
4039
import static org.hamcrest.Matchers.greaterThan;
4140
import static org.hamcrest.Matchers.lessThan;
@@ -97,13 +96,6 @@ public OutputStream apply(OutputStream stream) {
9796
@After
9897
public void tearDown() {
9998
Utils.deleteRecursively(tempDir);
100-
// This call is a workaround for SPARK-7660, a snappy-java bug which is exposed by this test
101-
// suite. Clearing the cached buffer allocator's pool of reusable buffers masks this bug,
102-
// preventing a test failure in JavaAPISuite that would otherwise occur. The underlying bug
103-
// needs to be fixed, but in the meantime this workaround avoids spurious Jenkins failures.
104-
synchronized (CachedBufferAllocator.class) {
105-
CachedBufferAllocator.queueTable.clear();
106-
}
10799
final long leakedMemory = taskMemoryManager.cleanUpAllAllocatedMemory();
108100
if (leakedMemory != 0) {
109101
fail("Test leaked " + leakedMemory + " bytes of managed memory");

ec2/spark_ec2.py

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -48,7 +48,7 @@
4848
from urllib.request import urlopen, Request
4949
from urllib.error import HTTPError
5050

51-
SPARK_EC2_VERSION = "1.2.1"
51+
SPARK_EC2_VERSION = "1.3.1"
5252
SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__))
5353

5454
VALID_SPARK_VERSIONS = set([
@@ -65,6 +65,8 @@
6565
"1.1.1",
6666
"1.2.0",
6767
"1.2.1",
68+
"1.3.0",
69+
"1.3.1",
6870
])
6971

7072
SPARK_TACHYON_MAP = {
@@ -75,6 +77,8 @@
7577
"1.1.1": "0.5.0",
7678
"1.2.0": "0.5.0",
7779
"1.2.1": "0.5.0",
80+
"1.3.0": "0.5.0",
81+
"1.3.1": "0.5.0",
7882
}
7983

8084
DEFAULT_SPARK_VERSION = SPARK_EC2_VERSION

0 commit comments

Comments
 (0)