Skip to content

Commit 62a7c28

Browse files
holdenkAndrew Or
authored andcommitted
[SPARK-13164][CORE] Replace deprecated synchronized buffer in core
Building with scala 2.11 results in the warning trait SynchronizedBuffer in package mutable is deprecated: Synchronization via traits is deprecated as it is inherently unreliable. Consider java.util.concurrent.ConcurrentLinkedQueue as an alternative. Investigation shows we are already using ConcurrentLinkedQueue in other locations so switch our uses of SynchronizedBuffer to ConcurrentLinkedQueue. Author: Holden Karau <[email protected]> Closes #11059 from holdenk/SPARK-13164-replace-deprecated-synchronized-buffer-in-core.
1 parent 2eaeafe commit 62a7c28

File tree

4 files changed

+40
-39
lines changed

4 files changed

+40
-39
lines changed

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

Lines changed: 12 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -18,9 +18,9 @@
1818
package org.apache.spark
1919

2020
import java.lang.ref.{ReferenceQueue, WeakReference}
21-
import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
21+
import java.util.concurrent.{ConcurrentLinkedQueue, ScheduledExecutorService, TimeUnit}
2222

23-
import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer}
23+
import scala.collection.JavaConverters._
2424

2525
import org.apache.spark.broadcast.Broadcast
2626
import org.apache.spark.rdd.{RDD, ReliableRDDCheckpointData}
@@ -57,13 +57,11 @@ private class CleanupTaskWeakReference(
5757
*/
5858
private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
5959

60-
private val referenceBuffer = new ArrayBuffer[CleanupTaskWeakReference]
61-
with SynchronizedBuffer[CleanupTaskWeakReference]
60+
private val referenceBuffer = new ConcurrentLinkedQueue[CleanupTaskWeakReference]()
6261

6362
private val referenceQueue = new ReferenceQueue[AnyRef]
6463

65-
private val listeners = new ArrayBuffer[CleanerListener]
66-
with SynchronizedBuffer[CleanerListener]
64+
private val listeners = new ConcurrentLinkedQueue[CleanerListener]()
6765

6866
private val cleaningThread = new Thread() { override def run() { keepCleaning() }}
6967

@@ -111,7 +109,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
111109

112110
/** Attach a listener object to get information of when objects are cleaned. */
113111
def attachListener(listener: CleanerListener): Unit = {
114-
listeners += listener
112+
listeners.add(listener)
115113
}
116114

117115
/** Start the cleaner. */
@@ -166,7 +164,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
166164

167165
/** Register an object for cleanup. */
168166
private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask): Unit = {
169-
referenceBuffer += new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue)
167+
referenceBuffer.add(new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue))
170168
}
171169

172170
/** Keep cleaning RDD, shuffle, and broadcast state. */
@@ -179,7 +177,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
179177
synchronized {
180178
reference.map(_.task).foreach { task =>
181179
logDebug("Got cleaning task " + task)
182-
referenceBuffer -= reference.get
180+
referenceBuffer.remove(reference.get)
183181
task match {
184182
case CleanRDD(rddId) =>
185183
doCleanupRDD(rddId, blocking = blockOnCleanupTasks)
@@ -206,7 +204,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
206204
try {
207205
logDebug("Cleaning RDD " + rddId)
208206
sc.unpersistRDD(rddId, blocking)
209-
listeners.foreach(_.rddCleaned(rddId))
207+
listeners.asScala.foreach(_.rddCleaned(rddId))
210208
logInfo("Cleaned RDD " + rddId)
211209
} catch {
212210
case e: Exception => logError("Error cleaning RDD " + rddId, e)
@@ -219,7 +217,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
219217
logDebug("Cleaning shuffle " + shuffleId)
220218
mapOutputTrackerMaster.unregisterShuffle(shuffleId)
221219
blockManagerMaster.removeShuffle(shuffleId, blocking)
222-
listeners.foreach(_.shuffleCleaned(shuffleId))
220+
listeners.asScala.foreach(_.shuffleCleaned(shuffleId))
223221
logInfo("Cleaned shuffle " + shuffleId)
224222
} catch {
225223
case e: Exception => logError("Error cleaning shuffle " + shuffleId, e)
@@ -231,7 +229,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
231229
try {
232230
logDebug(s"Cleaning broadcast $broadcastId")
233231
broadcastManager.unbroadcast(broadcastId, true, blocking)
234-
listeners.foreach(_.broadcastCleaned(broadcastId))
232+
listeners.asScala.foreach(_.broadcastCleaned(broadcastId))
235233
logDebug(s"Cleaned broadcast $broadcastId")
236234
} catch {
237235
case e: Exception => logError("Error cleaning broadcast " + broadcastId, e)
@@ -243,7 +241,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
243241
try {
244242
logDebug("Cleaning accumulator " + accId)
245243
Accumulators.remove(accId)
246-
listeners.foreach(_.accumCleaned(accId))
244+
listeners.asScala.foreach(_.accumCleaned(accId))
247245
logInfo("Cleaned accumulator " + accId)
248246
} catch {
249247
case e: Exception => logError("Error cleaning accumulator " + accId, e)
@@ -258,7 +256,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging {
258256
try {
259257
logDebug("Cleaning rdd checkpoint data " + rddId)
260258
ReliableRDDCheckpointData.cleanCheckpoint(sc, rddId)
261-
listeners.foreach(_.checkpointCleaned(rddId))
259+
listeners.asScala.foreach(_.checkpointCleaned(rddId))
262260
logInfo("Cleaned rdd checkpoint data " + rddId)
263261
}
264262
catch {

core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala

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

1818
package org.apache.spark.deploy.client
1919

20-
import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer}
20+
import java.util.concurrent.ConcurrentLinkedQueue
21+
22+
import scala.collection.JavaConverters._
2123
import scala.concurrent.duration._
2224

2325
import org.scalatest.BeforeAndAfterAll
@@ -165,14 +167,14 @@ class AppClientSuite extends SparkFunSuite with LocalSparkContext with BeforeAnd
165167

166168
/** Application Listener to collect events */
167169
private class AppClientCollector extends AppClientListener with Logging {
168-
val connectedIdList = new ArrayBuffer[String] with SynchronizedBuffer[String]
170+
val connectedIdList = new ConcurrentLinkedQueue[String]()
169171
@volatile var disconnectedCount: Int = 0
170-
val deadReasonList = new ArrayBuffer[String] with SynchronizedBuffer[String]
171-
val execAddedList = new ArrayBuffer[String] with SynchronizedBuffer[String]
172-
val execRemovedList = new ArrayBuffer[String] with SynchronizedBuffer[String]
172+
val deadReasonList = new ConcurrentLinkedQueue[String]()
173+
val execAddedList = new ConcurrentLinkedQueue[String]()
174+
val execRemovedList = new ConcurrentLinkedQueue[String]()
173175

174176
def connected(id: String): Unit = {
175-
connectedIdList += id
177+
connectedIdList.add(id)
176178
}
177179

178180
def disconnected(): Unit = {
@@ -182,7 +184,7 @@ class AppClientSuite extends SparkFunSuite with LocalSparkContext with BeforeAnd
182184
}
183185

184186
def dead(reason: String): Unit = {
185-
deadReasonList += reason
187+
deadReasonList.add(reason)
186188
}
187189

188190
def executorAdded(
@@ -191,11 +193,11 @@ class AppClientSuite extends SparkFunSuite with LocalSparkContext with BeforeAnd
191193
hostPort: String,
192194
cores: Int,
193195
memory: Int): Unit = {
194-
execAddedList += id
196+
execAddedList.add(id)
195197
}
196198

197199
def executorRemoved(id: String, message: String, exitStatus: Option[Int]): Unit = {
198-
execRemovedList += id
200+
execRemovedList.add(id)
199201
}
200202
}
201203

core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala

Lines changed: 12 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -20,9 +20,10 @@ package org.apache.spark.rpc
2020
import java.io.{File, NotSerializableException}
2121
import java.nio.charset.StandardCharsets.UTF_8
2222
import java.util.UUID
23-
import java.util.concurrent.{CountDownLatch, TimeoutException, TimeUnit}
23+
import java.util.concurrent.{ConcurrentLinkedQueue, CountDownLatch, TimeoutException, TimeUnit}
2424

2525
import scala.collection.mutable
26+
import scala.collection.JavaConverters._
2627
import scala.concurrent.Await
2728
import scala.concurrent.duration._
2829
import scala.language.postfixOps
@@ -490,30 +491,30 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
490491

491492
/**
492493
* Setup an [[RpcEndpoint]] to collect all network events.
493-
* @return the [[RpcEndpointRef]] and an `Seq` that contains network events.
494+
* @return the [[RpcEndpointRef]] and an `ConcurrentLinkedQueue` that contains network events.
494495
*/
495496
private def setupNetworkEndpoint(
496497
_env: RpcEnv,
497-
name: String): (RpcEndpointRef, Seq[(Any, Any)]) = {
498-
val events = new mutable.ArrayBuffer[(Any, Any)] with mutable.SynchronizedBuffer[(Any, Any)]
498+
name: String): (RpcEndpointRef, ConcurrentLinkedQueue[(Any, Any)]) = {
499+
val events = new ConcurrentLinkedQueue[(Any, Any)]
499500
val ref = _env.setupEndpoint("network-events-non-client", new ThreadSafeRpcEndpoint {
500501
override val rpcEnv = _env
501502

502503
override def receive: PartialFunction[Any, Unit] = {
503504
case "hello" =>
504-
case m => events += "receive" -> m
505+
case m => events.add("receive" -> m)
505506
}
506507

507508
override def onConnected(remoteAddress: RpcAddress): Unit = {
508-
events += "onConnected" -> remoteAddress
509+
events.add("onConnected" -> remoteAddress)
509510
}
510511

511512
override def onDisconnected(remoteAddress: RpcAddress): Unit = {
512-
events += "onDisconnected" -> remoteAddress
513+
events.add("onDisconnected" -> remoteAddress)
513514
}
514515

515516
override def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = {
516-
events += "onNetworkError" -> remoteAddress
517+
events.add("onNetworkError" -> remoteAddress)
517518
}
518519

519520
})
@@ -560,16 +561,16 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
560561

561562
eventually(timeout(5 seconds), interval(5 millis)) {
562563
// We don't know the exact client address but at least we can verify the message type
563-
assert(events.map(_._1).contains("onConnected"))
564+
assert(events.asScala.map(_._1).exists(_ == "onConnected"))
564565
}
565566

566567
clientEnv.shutdown()
567568
clientEnv.awaitTermination()
568569

569570
eventually(timeout(5 seconds), interval(5 millis)) {
570571
// We don't know the exact client address but at least we can verify the message type
571-
assert(events.map(_._1).contains("onConnected"))
572-
assert(events.map(_._1).contains("onDisconnected"))
572+
assert(events.asScala.map(_._1).exists(_ == "onConnected"))
573+
assert(events.asScala.map(_._1).exists(_ == "onDisconnected"))
573574
}
574575
} finally {
575576
clientEnv.shutdown()

core/src/test/scala/org/apache/spark/util/EventLoopSuite.scala

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -17,9 +17,9 @@
1717

1818
package org.apache.spark.util
1919

20-
import java.util.concurrent.CountDownLatch
20+
import java.util.concurrent.{ConcurrentLinkedQueue, CountDownLatch}
2121

22-
import scala.collection.mutable
22+
import scala.collection.JavaConverters._
2323
import scala.concurrent.duration._
2424
import scala.language.postfixOps
2525

@@ -31,19 +31,19 @@ import org.apache.spark.SparkFunSuite
3131
class EventLoopSuite extends SparkFunSuite with Timeouts {
3232

3333
test("EventLoop") {
34-
val buffer = new mutable.ArrayBuffer[Int] with mutable.SynchronizedBuffer[Int]
34+
val buffer = new ConcurrentLinkedQueue[Int]
3535
val eventLoop = new EventLoop[Int]("test") {
3636

3737
override def onReceive(event: Int): Unit = {
38-
buffer += event
38+
buffer.add(event)
3939
}
4040

4141
override def onError(e: Throwable): Unit = {}
4242
}
4343
eventLoop.start()
4444
(1 to 100).foreach(eventLoop.post)
4545
eventually(timeout(5 seconds), interval(5 millis)) {
46-
assert((1 to 100) === buffer.toSeq)
46+
assert((1 to 100) === buffer.asScala.toSeq)
4747
}
4848
eventLoop.stop()
4949
}

0 commit comments

Comments
 (0)