Skip to content

Commit 66d076f

Browse files
committed
Merge remote-tracking branch 'upstream/master' into dt-opt2
2 parents a0ed0da + 95470a0 commit 66d076f

File tree

54 files changed

+793
-450
lines changed

Some content is hidden

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

54 files changed

+793
-450
lines changed

core/src/main/scala/org/apache/spark/ContextCleaner.scala

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -66,10 +66,15 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
6666

6767
/**
6868
* Whether the cleaning thread will block on cleanup tasks.
69-
* This is set to true only for tests.
69+
*
70+
* Due to SPARK-3015, this is set to true by default. This is intended to be only a temporary
71+
* workaround for the issue, which is ultimately caused by the way the BlockManager actors
72+
* issue inter-dependent blocking Akka messages to each other at high frequencies. This happens,
73+
* for instance, when the driver performs a GC and cleans up all broadcast blocks that are no
74+
* longer in scope.
7075
*/
7176
private val blockOnCleanupTasks = sc.conf.getBoolean(
72-
"spark.cleaner.referenceTracking.blocking", false)
77+
"spark.cleaner.referenceTracking.blocking", true)
7378

7479
@volatile private var stopped = false
7580

@@ -174,9 +179,6 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
174179
private def blockManagerMaster = sc.env.blockManager.master
175180
private def broadcastManager = sc.env.broadcastManager
176181
private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster]
177-
178-
// Used for testing. These methods explicitly blocks until cleanup is completed
179-
// to ensure that more reliable testing.
180182
}
181183

182184
private object ContextCleaner {

core/src/main/scala/org/apache/spark/SparkEnv.scala

Lines changed: 11 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -210,12 +210,22 @@ object SparkEnv extends Logging {
210210
"MapOutputTracker",
211211
new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf))
212212

213+
// Let the user specify short names for shuffle managers
214+
val shortShuffleMgrNames = Map(
215+
"hash" -> "org.apache.spark.shuffle.hash.HashShuffleManager",
216+
"sort" -> "org.apache.spark.shuffle.sort.SortShuffleManager")
217+
val shuffleMgrName = conf.get("spark.shuffle.manager", "hash")
218+
val shuffleMgrClass = shortShuffleMgrNames.getOrElse(shuffleMgrName.toLowerCase, shuffleMgrName)
219+
val shuffleManager = instantiateClass[ShuffleManager](shuffleMgrClass)
220+
221+
val shuffleMemoryManager = new ShuffleMemoryManager(conf)
222+
213223
val blockManagerMaster = new BlockManagerMaster(registerOrLookup(
214224
"BlockManagerMaster",
215225
new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf)
216226

217227
val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster,
218-
serializer, conf, securityManager, mapOutputTracker)
228+
serializer, conf, securityManager, mapOutputTracker, shuffleManager)
219229

220230
val connectionManager = blockManager.connectionManager
221231

@@ -250,16 +260,6 @@ object SparkEnv extends Logging {
250260
"."
251261
}
252262

253-
// Let the user specify short names for shuffle managers
254-
val shortShuffleMgrNames = Map(
255-
"hash" -> "org.apache.spark.shuffle.hash.HashShuffleManager",
256-
"sort" -> "org.apache.spark.shuffle.sort.SortShuffleManager")
257-
val shuffleMgrName = conf.get("spark.shuffle.manager", "hash")
258-
val shuffleMgrClass = shortShuffleMgrNames.getOrElse(shuffleMgrName.toLowerCase, shuffleMgrName)
259-
val shuffleManager = instantiateClass[ShuffleManager](shuffleMgrClass)
260-
261-
val shuffleMemoryManager = new ShuffleMemoryManager(conf)
262-
263263
// Warn about deprecated spark.cache.class property
264264
if (conf.contains("spark.cache.class")) {
265265
logWarning("The spark.cache.class property is no longer being used! Specify storage " +

core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -315,6 +315,14 @@ private[spark] object PythonRDD extends Logging {
315315
JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism))
316316
}
317317

318+
def readBroadcastFromFile(sc: JavaSparkContext, filename: String): Broadcast[Array[Byte]] = {
319+
val file = new DataInputStream(new FileInputStream(filename))
320+
val length = file.readInt()
321+
val obj = new Array[Byte](length)
322+
file.readFully(obj)
323+
sc.broadcast(obj)
324+
}
325+
318326
def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) {
319327
// The right way to implement this would be to use TypeTags to get the full
320328
// type of T. Since I don't want to introduce breaking changes throughout the

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

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -99,6 +99,9 @@ private[spark] class Executor(
9999
private val urlClassLoader = createClassLoader()
100100
private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader)
101101

102+
// Set the classloader for serializer
103+
env.serializer.setDefaultClassLoader(urlClassLoader)
104+
102105
// Akka's message frame size. If task result is bigger than this, we use the block manager
103106
// to send the result back.
104107
private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf)

core/src/main/scala/org/apache/spark/network/ConnectionManager.scala

Lines changed: 36 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ import java.nio._
2222
import java.nio.channels._
2323
import java.nio.channels.spi._
2424
import java.net._
25+
import java.util.{Timer, TimerTask}
2526
import java.util.concurrent.atomic.AtomicInteger
2627

2728
import java.util.concurrent.{LinkedBlockingDeque, TimeUnit, ThreadPoolExecutor}
@@ -61,17 +62,17 @@ private[spark] class ConnectionManager(
6162
var ackMessage: Option[Message] = None
6263

6364
def markDone(ackMessage: Option[Message]) {
64-
this.synchronized {
65-
this.ackMessage = ackMessage
66-
completionHandler(this)
67-
}
65+
this.ackMessage = ackMessage
66+
completionHandler(this)
6867
}
6968
}
7069

7170
private val selector = SelectorProvider.provider.openSelector()
71+
private val ackTimeoutMonitor = new Timer("AckTimeoutMonitor", true)
7272

7373
// default to 30 second timeout waiting for authentication
7474
private val authTimeout = conf.getInt("spark.core.connection.auth.wait.timeout", 30)
75+
private val ackTimeout = conf.getInt("spark.core.connection.ack.wait.timeout", 60)
7576

7677
private val handleMessageExecutor = new ThreadPoolExecutor(
7778
conf.getInt("spark.core.connection.handler.threads.min", 20),
@@ -652,19 +653,27 @@ private[spark] class ConnectionManager(
652653
}
653654
}
654655
if (bufferMessage.hasAckId()) {
655-
val sentMessageStatus = messageStatuses.synchronized {
656+
messageStatuses.synchronized {
656657
messageStatuses.get(bufferMessage.ackId) match {
657658
case Some(status) => {
658659
messageStatuses -= bufferMessage.ackId
659-
status
660+
status.markDone(Some(message))
660661
}
661662
case None => {
662-
throw new Exception("Could not find reference for received ack message " +
663-
message.id)
663+
/**
664+
* We can fall down on this code because of following 2 cases
665+
*
666+
* (1) Invalid ack sent due to buggy code.
667+
*
668+
* (2) Late-arriving ack for a SendMessageStatus
669+
* To avoid unwilling late-arriving ack
670+
* caused by long pause like GC, you can set
671+
* larger value than default to spark.core.connection.ack.wait.timeout
672+
*/
673+
logWarning(s"Could not find reference for received ack Message ${message.id}")
664674
}
665675
}
666676
}
667-
sentMessageStatus.markDone(Some(message))
668677
} else {
669678
var ackMessage : Option[Message] = None
670679
try {
@@ -836,9 +845,23 @@ private[spark] class ConnectionManager(
836845
def sendMessageReliably(connectionManagerId: ConnectionManagerId, message: Message)
837846
: Future[Message] = {
838847
val promise = Promise[Message]()
848+
849+
val timeoutTask = new TimerTask {
850+
override def run(): Unit = {
851+
messageStatuses.synchronized {
852+
messageStatuses.remove(message.id).foreach ( s => {
853+
promise.failure(
854+
new IOException(s"sendMessageReliably failed because ack " +
855+
"was not received within ${ackTimeout} sec"))
856+
})
857+
}
858+
}
859+
}
860+
839861
val status = new MessageStatus(message, connectionManagerId, s => {
862+
timeoutTask.cancel()
840863
s.ackMessage match {
841-
case None => // Indicates a failure where we either never sent or never got ACK'd
864+
case None => // Indicates a failure where we either never sent or never got ACK'd
842865
promise.failure(new IOException("sendMessageReliably failed without being ACK'd"))
843866
case Some(ackMessage) =>
844867
if (ackMessage.hasError) {
@@ -852,6 +875,8 @@ private[spark] class ConnectionManager(
852875
messageStatuses.synchronized {
853876
messageStatuses += ((message.id, status))
854877
}
878+
879+
ackTimeoutMonitor.schedule(timeoutTask, ackTimeout * 1000)
855880
sendMessage(connectionManagerId, message)
856881
promise.future
857882
}
@@ -861,6 +886,7 @@ private[spark] class ConnectionManager(
861886
}
862887

863888
def stop() {
889+
ackTimeoutMonitor.cancel()
864890
selectorThread.interrupt()
865891
selectorThread.join()
866892
selector.close()

core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala

Lines changed: 15 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -63,32 +63,35 @@ extends DeserializationStream {
6363
def close() { objIn.close() }
6464
}
6565

66-
private[spark] class JavaSerializerInstance(counterReset: Int) extends SerializerInstance {
67-
def serialize[T: ClassTag](t: T): ByteBuffer = {
66+
67+
private[spark] class JavaSerializerInstance(counterReset: Int, defaultClassLoader: ClassLoader)
68+
extends SerializerInstance {
69+
70+
override def serialize[T: ClassTag](t: T): ByteBuffer = {
6871
val bos = new ByteArrayOutputStream()
6972
val out = serializeStream(bos)
7073
out.writeObject(t)
7174
out.close()
7275
ByteBuffer.wrap(bos.toByteArray)
7376
}
7477

75-
def deserialize[T: ClassTag](bytes: ByteBuffer): T = {
78+
override def deserialize[T: ClassTag](bytes: ByteBuffer): T = {
7679
val bis = new ByteBufferInputStream(bytes)
7780
val in = deserializeStream(bis)
78-
in.readObject().asInstanceOf[T]
81+
in.readObject()
7982
}
8083

81-
def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = {
84+
override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = {
8285
val bis = new ByteBufferInputStream(bytes)
8386
val in = deserializeStream(bis, loader)
84-
in.readObject().asInstanceOf[T]
87+
in.readObject()
8588
}
8689

87-
def serializeStream(s: OutputStream): SerializationStream = {
90+
override def serializeStream(s: OutputStream): SerializationStream = {
8891
new JavaSerializationStream(s, counterReset)
8992
}
9093

91-
def deserializeStream(s: InputStream): DeserializationStream = {
94+
override def deserializeStream(s: InputStream): DeserializationStream = {
9295
new JavaDeserializationStream(s, Utils.getContextOrSparkClassLoader)
9396
}
9497

@@ -109,7 +112,10 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize
109112
class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable {
110113
private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 100)
111114

112-
def newInstance(): SerializerInstance = new JavaSerializerInstance(counterReset)
115+
override def newInstance(): SerializerInstance = {
116+
val classLoader = defaultClassLoader.getOrElse(Thread.currentThread.getContextClassLoader)
117+
new JavaSerializerInstance(counterReset, classLoader)
118+
}
113119

114120
override def writeExternal(out: ObjectOutput) {
115121
out.writeInt(counterReset)

core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala

Lines changed: 24 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -61,7 +61,9 @@ class KryoSerializer(conf: SparkConf)
6161
val instantiator = new EmptyScalaKryoInstantiator
6262
val kryo = instantiator.newKryo()
6363
kryo.setRegistrationRequired(registrationRequired)
64-
val classLoader = Thread.currentThread.getContextClassLoader
64+
65+
val oldClassLoader = Thread.currentThread.getContextClassLoader
66+
val classLoader = defaultClassLoader.getOrElse(Thread.currentThread.getContextClassLoader)
6567

6668
// Allow disabling Kryo reference tracking if user knows their object graphs don't have loops.
6769
// Do this before we invoke the user registrator so the user registrator can override this.
@@ -84,10 +86,15 @@ class KryoSerializer(conf: SparkConf)
8486
try {
8587
val reg = Class.forName(regCls, true, classLoader).newInstance()
8688
.asInstanceOf[KryoRegistrator]
89+
90+
// Use the default classloader when calling the user registrator.
91+
Thread.currentThread.setContextClassLoader(classLoader)
8792
reg.registerClasses(kryo)
8893
} catch {
89-
case e: Exception =>
94+
case e: Exception =>
9095
throw new SparkException(s"Failed to invoke $regCls", e)
96+
} finally {
97+
Thread.currentThread.setContextClassLoader(oldClassLoader)
9198
}
9299
}
93100

@@ -99,7 +106,7 @@ class KryoSerializer(conf: SparkConf)
99106
kryo
100107
}
101108

102-
def newInstance(): SerializerInstance = {
109+
override def newInstance(): SerializerInstance = {
103110
new KryoSerializerInstance(this)
104111
}
105112
}
@@ -108,20 +115,20 @@ private[spark]
108115
class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream {
109116
val output = new KryoOutput(outStream)
110117

111-
def writeObject[T: ClassTag](t: T): SerializationStream = {
118+
override def writeObject[T: ClassTag](t: T): SerializationStream = {
112119
kryo.writeClassAndObject(output, t)
113120
this
114121
}
115122

116-
def flush() { output.flush() }
117-
def close() { output.close() }
123+
override def flush() { output.flush() }
124+
override def close() { output.close() }
118125
}
119126

120127
private[spark]
121128
class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream {
122-
val input = new KryoInput(inStream)
129+
private val input = new KryoInput(inStream)
123130

124-
def readObject[T: ClassTag](): T = {
131+
override def readObject[T: ClassTag](): T = {
125132
try {
126133
kryo.readClassAndObject(input).asInstanceOf[T]
127134
} catch {
@@ -131,31 +138,31 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser
131138
}
132139
}
133140

134-
def close() {
141+
override def close() {
135142
// Kryo's Input automatically closes the input stream it is using.
136143
input.close()
137144
}
138145
}
139146

140147
private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance {
141-
val kryo = ks.newKryo()
148+
private val kryo = ks.newKryo()
142149

143150
// Make these lazy vals to avoid creating a buffer unless we use them
144-
lazy val output = ks.newKryoOutput()
145-
lazy val input = new KryoInput()
151+
private lazy val output = ks.newKryoOutput()
152+
private lazy val input = new KryoInput()
146153

147-
def serialize[T: ClassTag](t: T): ByteBuffer = {
154+
override def serialize[T: ClassTag](t: T): ByteBuffer = {
148155
output.clear()
149156
kryo.writeClassAndObject(output, t)
150157
ByteBuffer.wrap(output.toBytes)
151158
}
152159

153-
def deserialize[T: ClassTag](bytes: ByteBuffer): T = {
160+
override def deserialize[T: ClassTag](bytes: ByteBuffer): T = {
154161
input.setBuffer(bytes.array)
155162
kryo.readClassAndObject(input).asInstanceOf[T]
156163
}
157164

158-
def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = {
165+
override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = {
159166
val oldClassLoader = kryo.getClassLoader
160167
kryo.setClassLoader(loader)
161168
input.setBuffer(bytes.array)
@@ -164,11 +171,11 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ
164171
obj
165172
}
166173

167-
def serializeStream(s: OutputStream): SerializationStream = {
174+
override def serializeStream(s: OutputStream): SerializationStream = {
168175
new KryoSerializationStream(kryo, s)
169176
}
170177

171-
def deserializeStream(s: InputStream): DeserializationStream = {
178+
override def deserializeStream(s: InputStream): DeserializationStream = {
172179
new KryoDeserializationStream(kryo, s)
173180
}
174181
}

0 commit comments

Comments
 (0)