Skip to content

Commit 7aca0dd

Browse files
committed
[SPARK-29296][BUILD][CORE] Remove use of .par to make 2.13 support easier; add scala-2.13 profile to enable pulling in par collections library separately, for the future
### What changes were proposed in this pull request? Scala 2.13 removes the parallel collections classes to a separate library, so first, this establishes a `scala-2.13` profile to bring it back, for future use. However the library enables use of `.par` implicit conversions via a new class that is not in 2.12, which makes cross-building hard. This implements a suggested workaround from scala/scala-parallel-collections#22 to avoid `.par` entirely. ### Why are the changes needed? To compile for 2.13 and later to work with 2.13. ### Does this PR introduce any user-facing change? Should not, no. ### How was this patch tested? Existing tests. Closes #25980 from srowen/SPARK-29296. Authored-by: Sean Owen <[email protected]> Signed-off-by: Sean Owen <[email protected]>
1 parent 2bc3fff commit 7aca0dd

File tree

12 files changed

+78
-11
lines changed

12 files changed

+78
-11
lines changed

core/pom.xml

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -556,6 +556,15 @@
556556
</plugins>
557557
</build>
558558
</profile>
559+
<profile>
560+
<id>scala-2.13</id>
561+
<dependencies>
562+
<dependency>
563+
<groupId>org.scala-lang.modules</groupId>
564+
<artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
565+
</dependency>
566+
</dependencies>
567+
</profile>
559568
</profiles>
560569

561570
</project>

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

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ import java.io.{IOException, ObjectOutputStream}
2121

2222
import scala.collection.mutable.ArrayBuffer
2323
import scala.collection.parallel.ForkJoinTaskSupport
24+
import scala.collection.parallel.immutable.ParVector
2425
import scala.reflect.ClassTag
2526

2627
import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext}
@@ -75,13 +76,13 @@ class UnionRDD[T: ClassTag](
7576

7677
override def getPartitions: Array[Partition] = {
7778
val parRDDs = if (isPartitionListingParallel) {
78-
val parArray = rdds.par
79+
val parArray = new ParVector(rdds.toVector)
7980
parArray.tasksupport = UnionRDD.partitionEvalTaskSupport
8081
parArray
8182
} else {
8283
rdds
8384
}
84-
val array = new Array[Partition](parRDDs.map(_.partitions.length).seq.sum)
85+
val array = new Array[Partition](parRDDs.map(_.partitions.length).sum)
8586
var pos = 0
8687
for ((rdd, rddIndex) <- rdds.zipWithIndex; split <- rdd.partitions) {
8788
array(pos) = new UnionPartition(pos, rdd, rddIndex, split.index)

pom.xml

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2992,6 +2992,19 @@
29922992
<profile>
29932993
<id>scala-2.12</id>
29942994
</profile>
2995+
2996+
<profile>
2997+
<id>scala-2.13</id>
2998+
<dependencyManagement>
2999+
<dependencies>
3000+
<dependency>
3001+
<groupId>org.scala-lang.modules</groupId>
3002+
<artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
3003+
<version>0.2.0</version>
3004+
</dependency>
3005+
</dependencies>
3006+
</dependencyManagement>
3007+
</profile>
29953008

29963009
<!--
29973010
This is a profile to enable the use of the ASF snapshot and staging repositories

sql/catalyst/pom.xml

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -170,4 +170,16 @@
170170
</plugin>
171171
</plugins>
172172
</build>
173+
174+
<profiles>
175+
<profile>
176+
<id>scala-2.13</id>
177+
<dependencies>
178+
<dependency>
179+
<groupId>org.scala-lang.modules</groupId>
180+
<artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
181+
</dependency>
182+
</dependencies>
183+
</profile>
184+
</profiles>
173185
</project>

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,8 @@ import java.sql.{Date, Timestamp}
2121
import java.util.{Calendar, TimeZone}
2222
import java.util.concurrent.TimeUnit._
2323

24+
import scala.collection.parallel.immutable.ParVector
25+
2426
import org.apache.spark.SparkFunSuite
2527
import org.apache.spark.sql.Row
2628
import org.apache.spark.sql.catalyst.InternalRow
@@ -113,7 +115,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper {
113115
}
114116

115117
test("cast string to timestamp") {
116-
ALL_TIMEZONES.par.foreach { tz =>
118+
new ParVector(ALL_TIMEZONES.toVector).foreach { tz =>
117119
def checkCastStringToTimestamp(str: String, expected: Timestamp): Unit = {
118120
checkEvaluation(cast(Literal(str), TimestampType, Option(tz.getID)), expected)
119121
}

sql/core/pom.xml

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -215,4 +215,16 @@
215215
</plugin>
216216
</plugins>
217217
</build>
218+
219+
<profiles>
220+
<profile>
221+
<id>scala-2.13</id>
222+
<dependencies>
223+
<dependency>
224+
<groupId>org.scala-lang.modules</groupId>
225+
<artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
226+
</dependency>
227+
</dependencies>
228+
</profile>
229+
</profiles>
218230
</project>

sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ import java.util.concurrent.TimeUnit._
2222

2323
import scala.collection.{GenMap, GenSeq}
2424
import scala.collection.parallel.ForkJoinTaskSupport
25+
import scala.collection.parallel.immutable.ParVector
2526
import scala.util.control.NonFatal
2627

2728
import org.apache.hadoop.conf.Configuration
@@ -684,7 +685,7 @@ case class AlterTableRecoverPartitionsCommand(
684685
val statusPar: GenSeq[FileStatus] =
685686
if (partitionNames.length > 1 && statuses.length > threshold || partitionNames.length > 2) {
686687
// parallelize the list of partitions here, then we can have better parallelism later.
687-
val parArray = statuses.par
688+
val parArray = new ParVector(statuses.toVector)
688689
parArray.tasksupport = evalTaskSupport
689690
parArray
690691
} else {

sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,8 @@ import java.net.{MalformedURLException, URL}
2222
import java.sql.{Date, Timestamp}
2323
import java.util.concurrent.atomic.AtomicBoolean
2424

25+
import scala.collection.parallel.immutable.ParVector
26+
2527
import org.apache.spark.{AccumulatorSuite, SparkException}
2628
import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart}
2729
import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation
@@ -169,7 +171,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession {
169171
"org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection")
170172

171173
withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") {
172-
spark.sessionState.functionRegistry.listFunction().par.foreach { funcId =>
174+
val parFuncs = new ParVector(spark.sessionState.functionRegistry.listFunction().toVector)
175+
parFuncs.foreach { funcId =>
173176
// Examples can change settings. We clone the session to prevent tests clashing.
174177
val clonedSpark = spark.cloneSession()
175178
val info = clonedSpark.sessionState.catalog.lookupFunctionInfo(funcId)

sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala

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

1818
package org.apache.spark.sql.execution
1919

20-
import java.util.Properties
20+
import scala.collection.parallel.immutable.ParRange
2121

2222
import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
2323
import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart}
@@ -46,7 +46,7 @@ class SQLExecutionSuite extends SparkFunSuite {
4646
import spark.implicits._
4747
try {
4848
// Should not throw IllegalArgumentException
49-
(1 to 100).par.foreach { _ =>
49+
new ParRange(1 to 100).foreach { _ =>
5050
spark.sparkContext.parallelize(1 to 5).map { i => (i, i) }.toDF("a", "b").count()
5151
}
5252
} finally {

streaming/pom.xml

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -131,4 +131,16 @@
131131
</plugin>
132132
</plugins>
133133
</build>
134+
135+
<profiles>
136+
<profile>
137+
<id>scala-2.13</id>
138+
<dependencies>
139+
<dependency>
140+
<groupId>org.scala-lang.modules</groupId>
141+
<artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
142+
</dependency>
143+
</dependencies>
144+
</profile>
145+
</profiles>
134146
</project>

streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ package org.apache.spark.streaming
2020
import java.io.{IOException, ObjectInputStream, ObjectOutputStream}
2121

2222
import scala.collection.mutable.ArrayBuffer
23+
import scala.collection.parallel.immutable.ParVector
2324

2425
import org.apache.spark.internal.Logging
2526
import org.apache.spark.streaming.dstream.{DStream, InputDStream, ReceiverInputDStream}
@@ -50,8 +51,8 @@ final private[streaming] class DStreamGraph extends Serializable with Logging {
5051
outputStreams.foreach(_.remember(rememberDuration))
5152
outputStreams.foreach(_.validateAtStart())
5253
numReceivers = inputStreams.count(_.isInstanceOf[ReceiverInputDStream[_]])
53-
inputStreamNameAndID = inputStreams.map(is => (is.name, is.id))
54-
inputStreams.par.foreach(_.start())
54+
inputStreamNameAndID = inputStreams.map(is => (is.name, is.id)).toSeq
55+
new ParVector(inputStreams.toVector).foreach(_.start())
5556
}
5657
}
5758

@@ -61,7 +62,7 @@ final private[streaming] class DStreamGraph extends Serializable with Logging {
6162

6263
def stop(): Unit = {
6364
this.synchronized {
64-
inputStreams.par.foreach(_.stop())
65+
new ParVector(inputStreams.toVector).foreach(_.stop())
6566
}
6667
}
6768

streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ import java.util.concurrent.RejectedExecutionException
2424
import scala.collection.JavaConverters._
2525
import scala.collection.mutable.ArrayBuffer
2626
import scala.collection.parallel.ExecutionContextTaskSupport
27+
import scala.collection.parallel.immutable.ParVector
2728
import scala.concurrent.{Await, ExecutionContext, Future}
2829

2930
import org.apache.hadoop.conf.Configuration
@@ -313,7 +314,7 @@ private[streaming] object FileBasedWriteAheadLog {
313314
val groupSize = taskSupport.parallelismLevel.max(8)
314315

315316
source.grouped(groupSize).flatMap { group =>
316-
val parallelCollection = group.par
317+
val parallelCollection = new ParVector(group.toVector)
317318
parallelCollection.tasksupport = taskSupport
318319
parallelCollection.map(handler)
319320
}.flatten

0 commit comments

Comments
 (0)