Skip to content

Commit

Permalink
[SPARK-29401][CORE][ML][SQL][GRAPHX][TESTS] Replace calls to .paralle…
Browse files Browse the repository at this point in the history
…lize Arrays of tuples, ambiguous in Scala 2.13, with Seqs of tuples

### What changes were proposed in this pull request?

Invocations like `sc.parallelize(Array((1,2)))` cause a compile error in 2.13, like:
```
[ERROR] [Error] /Users/seanowen/Documents/spark_2.13/core/src/test/scala/org/apache/spark/ShuffleSuite.scala:47: overloaded method value apply with alternatives:
  (x: Unit,xs: Unit*)Array[Unit] <and>
  (x: Double,xs: Double*)Array[Double] <and>
  (x: Float,xs: Float*)Array[Float] <and>
  (x: Long,xs: Long*)Array[Long] <and>
  (x: Int,xs: Int*)Array[Int] <and>
  (x: Char,xs: Char*)Array[Char] <and>
  (x: Short,xs: Short*)Array[Short] <and>
  (x: Byte,xs: Byte*)Array[Byte] <and>
  (x: Boolean,xs: Boolean*)Array[Boolean]
 cannot be applied to ((Int, Int), (Int, Int), (Int, Int), (Int, Int))
```
Using a `Seq` instead appears to resolve it, and is effectively equivalent.

### Why are the changes needed?

To better cross-build for 2.13.

### Does this PR introduce any user-facing change?

None.

### How was this patch tested?

Existing tests.

Closes apache#26062 from srowen/SPARK-29401.

Authored-by: Sean Owen <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
  • Loading branch information
srowen authored and dongjoon-hyun committed Oct 9, 2019
1 parent 2d871ad commit ee83d09
Show file tree
Hide file tree
Showing 11 changed files with 77 additions and 77 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex

test("simple groupByKey") {
sc = new SparkContext(clusterUrl, "test")
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 5)
val pairs = sc.parallelize(Seq((1, 1), (1, 2), (1, 3), (2, 1)), 5)
val groups = pairs.groupByKey(5).collect()
assert(groups.size === 2)
val valuesFor1 = groups.find(_._1 == 1).get._2
Expand Down
6 changes: 3 additions & 3 deletions core/src/test/scala/org/apache/spark/FileSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -372,15 +372,15 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {

test ("prevent user from overwriting the empty directory (old Hadoop API)") {
sc = new SparkContext("local", "test")
val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1)
val randomRDD = sc.parallelize(Seq((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1)
intercept[FileAlreadyExistsException] {
randomRDD.saveAsTextFile(tempDir.getPath)
}
}

test ("prevent user from overwriting the non-empty directory (old Hadoop API)") {
sc = new SparkContext("local", "test")
val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1)
val randomRDD = sc.parallelize(Seq((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1)
randomRDD.saveAsTextFile(tempDir.getPath + "/output")
assert(new File(tempDir.getPath + "/output/part-00000").exists())
intercept[FileAlreadyExistsException] {
Expand All @@ -392,7 +392,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
val conf = new SparkConf()
conf.setAppName("test").setMaster("local").set("spark.hadoop.validateOutputSpecs", "false")
sc = new SparkContext(conf)
val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1)
val randomRDD = sc.parallelize(Seq((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1)
randomRDD.saveAsTextFile(tempDir.getPath + "/output")
assert(new File(tempDir.getPath + "/output/part-00000").exists())
randomRDD.saveAsTextFile(tempDir.getPath + "/output")
Expand Down
14 changes: 7 additions & 7 deletions core/src/test/scala/org/apache/spark/PartitioningSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -262,11 +262,11 @@ class PartitioningSuite extends SparkFunSuite with SharedSparkContext with Priva

test("defaultPartitioner") {
val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 150)
val rdd2 = sc.parallelize(Array((1, 2), (2, 3), (2, 4), (3, 4)))
val rdd2 = sc.parallelize(Seq((1, 2), (2, 3), (2, 4), (3, 4)))
.partitionBy(new HashPartitioner(10))
val rdd3 = sc.parallelize(Array((1, 6), (7, 8), (3, 10), (5, 12), (13, 14)))
val rdd3 = sc.parallelize(Seq((1, 6), (7, 8), (3, 10), (5, 12), (13, 14)))
.partitionBy(new HashPartitioner(100))
val rdd4 = sc.parallelize(Array((1, 2), (2, 3), (2, 4), (3, 4)))
val rdd4 = sc.parallelize(Seq((1, 2), (2, 3), (2, 4), (3, 4)))
.partitionBy(new HashPartitioner(9))
val rdd5 = sc.parallelize((1 to 10).map(x => (x, x)), 11)

Expand All @@ -289,14 +289,14 @@ class PartitioningSuite extends SparkFunSuite with SharedSparkContext with Priva
sc.conf.set("spark.default.parallelism", "4")

val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 150)
val rdd2 = sc.parallelize(Array((1, 2), (2, 3), (2, 4), (3, 4)))
val rdd2 = sc.parallelize(Seq((1, 2), (2, 3), (2, 4), (3, 4)))
.partitionBy(new HashPartitioner(10))
val rdd3 = sc.parallelize(Array((1, 6), (7, 8), (3, 10), (5, 12), (13, 14)))
val rdd3 = sc.parallelize(Seq((1, 6), (7, 8), (3, 10), (5, 12), (13, 14)))
.partitionBy(new HashPartitioner(100))
val rdd4 = sc.parallelize(Array((1, 2), (2, 3), (2, 4), (3, 4)))
val rdd4 = sc.parallelize(Seq((1, 2), (2, 3), (2, 4), (3, 4)))
.partitionBy(new HashPartitioner(9))
val rdd5 = sc.parallelize((1 to 10).map(x => (x, x)), 11)
val rdd6 = sc.parallelize(Array((1, 2), (2, 3), (2, 4), (3, 4)))
val rdd6 = sc.parallelize(Seq((1, 2), (2, 3), (2, 4), (3, 4)))
.partitionBy(new HashPartitioner(3))

val partitioner1 = Partitioner.defaultPartitioner(rdd1, rdd2)
Expand Down
2 changes: 1 addition & 1 deletion core/src/test/scala/org/apache/spark/ShuffleSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC
test("groupByKey without compression") {
val myConf = conf.clone().set(config.SHUFFLE_COMPRESS, false)
sc = new SparkContext("local", "test", myConf)
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4)
val pairs = sc.parallelize(Seq((1, 1), (1, 2), (1, 3), (2, 1)), 4)
val groups = pairs.groupByKey(4).collect()
assert(groups.size === 2)
val valuesFor1 = groups.find(_._1 == 1).get._2
Expand Down
Loading

0 comments on commit ee83d09

Please sign in to comment.