[SPARK-29401][CORE][ML][SQL][GRAPHX][TESTS] Replace calls to .parallelize 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 #26062 from srowen/SPARK-29401.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
This commit is contained in:
Sean Owen 2019-10-08 20:22:02 -07:00 committed by Dongjoon Hyun
parent 2d871ad0e7
commit ee83d09b53
11 changed files with 77 additions and 77 deletions

View file

@ -77,7 +77,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex
test("simple groupByKey") { test("simple groupByKey") {
sc = new SparkContext(clusterUrl, "test") 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() val groups = pairs.groupByKey(5).collect()
assert(groups.size === 2) assert(groups.size === 2)
val valuesFor1 = groups.find(_._1 == 1).get._2 val valuesFor1 = groups.find(_._1 == 1).get._2

View file

@ -372,7 +372,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
test ("prevent user from overwriting the empty directory (old Hadoop API)") { test ("prevent user from overwriting the empty directory (old Hadoop API)") {
sc = new SparkContext("local", "test") 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] { intercept[FileAlreadyExistsException] {
randomRDD.saveAsTextFile(tempDir.getPath) randomRDD.saveAsTextFile(tempDir.getPath)
} }
@ -380,7 +380,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
test ("prevent user from overwriting the non-empty directory (old Hadoop API)") { test ("prevent user from overwriting the non-empty directory (old Hadoop API)") {
sc = new SparkContext("local", "test") 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") randomRDD.saveAsTextFile(tempDir.getPath + "/output")
assert(new File(tempDir.getPath + "/output/part-00000").exists()) assert(new File(tempDir.getPath + "/output/part-00000").exists())
intercept[FileAlreadyExistsException] { intercept[FileAlreadyExistsException] {
@ -392,7 +392,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
val conf = new SparkConf() val conf = new SparkConf()
conf.setAppName("test").setMaster("local").set("spark.hadoop.validateOutputSpecs", "false") conf.setAppName("test").setMaster("local").set("spark.hadoop.validateOutputSpecs", "false")
sc = new SparkContext(conf) 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") randomRDD.saveAsTextFile(tempDir.getPath + "/output")
assert(new File(tempDir.getPath + "/output/part-00000").exists()) assert(new File(tempDir.getPath + "/output/part-00000").exists())
randomRDD.saveAsTextFile(tempDir.getPath + "/output") randomRDD.saveAsTextFile(tempDir.getPath + "/output")

View file

@ -262,11 +262,11 @@ class PartitioningSuite extends SparkFunSuite with SharedSparkContext with Priva
test("defaultPartitioner") { test("defaultPartitioner") {
val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 150) 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)) .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)) .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)) .partitionBy(new HashPartitioner(9))
val rdd5 = sc.parallelize((1 to 10).map(x => (x, x)), 11) val rdd5 = sc.parallelize((1 to 10).map(x => (x, x)), 11)
@ -289,14 +289,14 @@ class PartitioningSuite extends SparkFunSuite with SharedSparkContext with Priva
sc.conf.set("spark.default.parallelism", "4") sc.conf.set("spark.default.parallelism", "4")
val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 150) 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)) .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)) .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)) .partitionBy(new HashPartitioner(9))
val rdd5 = sc.parallelize((1 to 10).map(x => (x, x)), 11) 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)) .partitionBy(new HashPartitioner(3))
val partitioner1 = Partitioner.defaultPartitioner(rdd1, rdd2) val partitioner1 = Partitioner.defaultPartitioner(rdd1, rdd2)

View file

@ -44,7 +44,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC
test("groupByKey without compression") { test("groupByKey without compression") {
val myConf = conf.clone().set(config.SHUFFLE_COMPRESS, false) val myConf = conf.clone().set(config.SHUFFLE_COMPRESS, false)
sc = new SparkContext("local", "test", myConf) 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() val groups = pairs.groupByKey(4).collect()
assert(groups.size === 2) assert(groups.size === 2)
val valuesFor1 = groups.find(_._1 == 1).get._2 val valuesFor1 = groups.find(_._1 == 1).get._2

View file

@ -38,7 +38,7 @@ import org.apache.spark.util.Utils
class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
test("aggregateByKey") { test("aggregateByKey") {
val pairs = sc.parallelize(Array((1, 1), (1, 1), (3, 2), (5, 1), (5, 3)), 2) val pairs = sc.parallelize(Seq((1, 1), (1, 1), (3, 2), (5, 1), (5, 3)), 2)
val sets = pairs.aggregateByKey(new HashSet[Int]())(_ += _, _ ++= _).collect() val sets = pairs.aggregateByKey(new HashSet[Int]())(_ += _, _ ++= _).collect()
assert(sets.size === 3) assert(sets.size === 3)
@ -51,7 +51,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("groupByKey") { test("groupByKey") {
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) val pairs = sc.parallelize(Seq((1, 1), (1, 2), (1, 3), (2, 1)))
val groups = pairs.groupByKey().collect() val groups = pairs.groupByKey().collect()
assert(groups.size === 2) assert(groups.size === 2)
val valuesFor1 = groups.find(_._1 == 1).get._2 val valuesFor1 = groups.find(_._1 == 1).get._2
@ -61,7 +61,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("groupByKey with duplicates") { test("groupByKey with duplicates") {
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) val pairs = sc.parallelize(Seq((1, 1), (1, 2), (1, 3), (1, 1), (2, 1)))
val groups = pairs.groupByKey().collect() val groups = pairs.groupByKey().collect()
assert(groups.size === 2) assert(groups.size === 2)
val valuesFor1 = groups.find(_._1 == 1).get._2 val valuesFor1 = groups.find(_._1 == 1).get._2
@ -71,7 +71,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("groupByKey with negative key hash codes") { test("groupByKey with negative key hash codes") {
val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))) val pairs = sc.parallelize(Seq((-1, 1), (-1, 2), (-1, 3), (2, 1)))
val groups = pairs.groupByKey().collect() val groups = pairs.groupByKey().collect()
assert(groups.size === 2) assert(groups.size === 2)
val valuesForMinus1 = groups.find(_._1 == -1).get._2 val valuesForMinus1 = groups.find(_._1 == -1).get._2
@ -81,7 +81,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("groupByKey with many output partitions") { test("groupByKey with many output partitions") {
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) val pairs = sc.parallelize(Seq((1, 1), (1, 2), (1, 3), (2, 1)))
val groups = pairs.groupByKey(10).collect() val groups = pairs.groupByKey(10).collect()
assert(groups.size === 2) assert(groups.size === 2)
val valuesFor1 = groups.find(_._1 == 1).get._2 val valuesFor1 = groups.find(_._1 == 1).get._2
@ -170,13 +170,13 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("reduceByKey") { test("reduceByKey") {
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) val pairs = sc.parallelize(Seq((1, 1), (1, 2), (1, 3), (1, 1), (2, 1)))
val sums = pairs.reduceByKey(_ + _).collect() val sums = pairs.reduceByKey(_ + _).collect()
assert(sums.toSet === Set((1, 7), (2, 1))) assert(sums.toSet === Set((1, 7), (2, 1)))
} }
test("reduceByKey with collectAsMap") { test("reduceByKey with collectAsMap") {
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) val pairs = sc.parallelize(Seq((1, 1), (1, 2), (1, 3), (1, 1), (2, 1)))
val sums = pairs.reduceByKey(_ + _).collectAsMap() val sums = pairs.reduceByKey(_ + _).collectAsMap()
assert(sums.size === 2) assert(sums.size === 2)
assert(sums(1) === 7) assert(sums(1) === 7)
@ -184,7 +184,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("reduceByKey with many output partitions") { test("reduceByKey with many output partitions") {
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) val pairs = sc.parallelize(Seq((1, 1), (1, 2), (1, 3), (1, 1), (2, 1)))
val sums = pairs.reduceByKey(_ + _, 10).collect() val sums = pairs.reduceByKey(_ + _, 10).collect()
assert(sums.toSet === Set((1, 7), (2, 1))) assert(sums.toSet === Set((1, 7), (2, 1)))
} }
@ -194,7 +194,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
def numPartitions = 2 def numPartitions = 2
def getPartition(key: Any) = key.asInstanceOf[Int] def getPartition(key: Any) = key.asInstanceOf[Int]
} }
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p) val pairs = sc.parallelize(Seq((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p)
val sums = pairs.reduceByKey(_ + _) val sums = pairs.reduceByKey(_ + _)
assert(sums.collect().toSet === Set((1, 4), (0, 1))) assert(sums.collect().toSet === Set((1, 4), (0, 1)))
assert(sums.partitioner === Some(p)) assert(sums.partitioner === Some(p))
@ -246,8 +246,8 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("join") { test("join") {
val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd1 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)))
val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val rdd2 = sc.parallelize(Seq((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w')))
val joined = rdd1.join(rdd2).collect() val joined = rdd1.join(rdd2).collect()
assert(joined.size === 4) assert(joined.size === 4)
assert(joined.toSet === Set( assert(joined.toSet === Set(
@ -259,8 +259,8 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("join all-to-all") { test("join all-to-all") {
val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))) val rdd1 = sc.parallelize(Seq((1, 1), (1, 2), (1, 3)))
val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))) val rdd2 = sc.parallelize(Seq((1, 'x'), (1, 'y')))
val joined = rdd1.join(rdd2).collect() val joined = rdd1.join(rdd2).collect()
assert(joined.size === 6) assert(joined.size === 6)
assert(joined.toSet === Set( assert(joined.toSet === Set(
@ -274,8 +274,8 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("leftOuterJoin") { test("leftOuterJoin") {
val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd1 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)))
val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val rdd2 = sc.parallelize(Seq((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w')))
val joined = rdd1.leftOuterJoin(rdd2).collect() val joined = rdd1.leftOuterJoin(rdd2).collect()
assert(joined.size === 5) assert(joined.size === 5)
assert(joined.toSet === Set( assert(joined.toSet === Set(
@ -292,7 +292,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
import scala.reflect.classTag import scala.reflect.classTag
val intPairCT = classTag[(Int, Int)] val intPairCT = classTag[(Int, Int)]
val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd1 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)))
val rdd2 = sc.emptyRDD[(Int, Int)](intPairCT) val rdd2 = sc.emptyRDD[(Int, Int)](intPairCT)
val joined = rdd1.cogroup(rdd2).collect() val joined = rdd1.cogroup(rdd2).collect()
@ -304,7 +304,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
import scala.reflect.classTag import scala.reflect.classTag
val intCT = classTag[Int] val intCT = classTag[Int]
val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd1 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)))
val rdd2 = sc.emptyRDD[Int](intCT).groupBy((x) => 5) val rdd2 = sc.emptyRDD[Int](intCT).groupBy((x) => 5)
val joined = rdd1.cogroup(rdd2).collect() val joined = rdd1.cogroup(rdd2).collect()
assert(joined.size > 0) assert(joined.size > 0)
@ -315,7 +315,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
"with an order of magnitude difference in number of partitions") { "with an order of magnitude difference in number of partitions") {
val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 1000) val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 1000)
val rdd2 = sc val rdd2 = sc
.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) .parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)))
.partitionBy(new HashPartitioner(10)) .partitionBy(new HashPartitioner(10))
val joined = rdd1.cogroup(rdd2) val joined = rdd1.cogroup(rdd2)
assert(joined.getNumPartitions == rdd1.getNumPartitions) assert(joined.getNumPartitions == rdd1.getNumPartitions)
@ -325,7 +325,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
test("cogroup between multiple RDD with number of partitions similar in order of magnitude") { test("cogroup between multiple RDD with number of partitions similar in order of magnitude") {
val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 20) val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 20)
val rdd2 = sc val rdd2 = sc
.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) .parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)))
.partitionBy(new HashPartitioner(10)) .partitionBy(new HashPartitioner(10))
val joined = rdd1.cogroup(rdd2) val joined = rdd1.cogroup(rdd2)
assert(joined.getNumPartitions == rdd2.getNumPartitions) assert(joined.getNumPartitions == rdd2.getNumPartitions)
@ -336,7 +336,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
try { try {
sc.conf.set("spark.default.parallelism", "4") sc.conf.set("spark.default.parallelism", "4")
val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 20) val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 20)
val rdd2 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1)), 10) val rdd2 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)), 10)
val joined = rdd1.cogroup(rdd2) val joined = rdd1.cogroup(rdd2)
assert(joined.getNumPartitions == sc.defaultParallelism) assert(joined.getNumPartitions == sc.defaultParallelism)
} finally { } finally {
@ -349,7 +349,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
try { try {
sc.conf.set("spark.default.parallelism", "4") sc.conf.set("spark.default.parallelism", "4")
val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 20) val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 20)
val rdd2 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd2 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)))
.partitionBy(new HashPartitioner(10)) .partitionBy(new HashPartitioner(10))
val joined = rdd1.cogroup(rdd2) val joined = rdd1.cogroup(rdd2)
assert(joined.getNumPartitions == rdd2.getNumPartitions) assert(joined.getNumPartitions == rdd2.getNumPartitions)
@ -364,7 +364,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
try { try {
sc.conf.set("spark.default.parallelism", "4") sc.conf.set("spark.default.parallelism", "4")
val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 1000) val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 1000)
val rdd2 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd2 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)))
.partitionBy(new HashPartitioner(10)) .partitionBy(new HashPartitioner(10))
val joined = rdd1.cogroup(rdd2) val joined = rdd1.cogroup(rdd2)
assert(joined.getNumPartitions == rdd2.getNumPartitions) assert(joined.getNumPartitions == rdd2.getNumPartitions)
@ -374,8 +374,8 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("rightOuterJoin") { test("rightOuterJoin") {
val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd1 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)))
val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val rdd2 = sc.parallelize(Seq((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w')))
val joined = rdd1.rightOuterJoin(rdd2).collect() val joined = rdd1.rightOuterJoin(rdd2).collect()
assert(joined.size === 5) assert(joined.size === 5)
assert(joined.toSet === Set( assert(joined.toSet === Set(
@ -388,8 +388,8 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("fullOuterJoin") { test("fullOuterJoin") {
val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd1 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)))
val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val rdd2 = sc.parallelize(Seq((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w')))
val joined = rdd1.fullOuterJoin(rdd2).collect() val joined = rdd1.fullOuterJoin(rdd2).collect()
assert(joined.size === 6) assert(joined.size === 6)
assert(joined.toSet === Set( assert(joined.toSet === Set(
@ -403,15 +403,15 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("join with no matches") { test("join with no matches") {
val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd1 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)))
val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) val rdd2 = sc.parallelize(Seq((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w')))
val joined = rdd1.join(rdd2).collect() val joined = rdd1.join(rdd2).collect()
assert(joined.size === 0) assert(joined.size === 0)
} }
test("join with many output partitions") { test("join with many output partitions") {
val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd1 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)))
val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val rdd2 = sc.parallelize(Seq((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w')))
val joined = rdd1.join(rdd2, 10).collect() val joined = rdd1.join(rdd2, 10).collect()
assert(joined.size === 4) assert(joined.size === 4)
assert(joined.toSet === Set( assert(joined.toSet === Set(
@ -423,8 +423,8 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("groupWith") { test("groupWith") {
val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd1 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)))
val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val rdd2 = sc.parallelize(Seq((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w')))
val joined = rdd1.groupWith(rdd2).collect() val joined = rdd1.groupWith(rdd2).collect()
assert(joined.size === 4) assert(joined.size === 4)
val joinedSet = joined.map(x => (x._1, (x._2._1.toList, x._2._2.toList))).toSet val joinedSet = joined.map(x => (x._1, (x._2._1.toList, x._2._2.toList))).toSet
@ -437,9 +437,9 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("groupWith3") { test("groupWith3") {
val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd1 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)))
val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val rdd2 = sc.parallelize(Seq((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w')))
val rdd3 = sc.parallelize(Array((1, 'a'), (3, 'b'), (4, 'c'), (4, 'd'))) val rdd3 = sc.parallelize(Seq((1, 'a'), (3, 'b'), (4, 'c'), (4, 'd')))
val joined = rdd1.groupWith(rdd2, rdd3).collect() val joined = rdd1.groupWith(rdd2, rdd3).collect()
assert(joined.size === 4) assert(joined.size === 4)
val joinedSet = joined.map(x => (x._1, val joinedSet = joined.map(x => (x._1,
@ -453,10 +453,10 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("groupWith4") { test("groupWith4") {
val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd1 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)))
val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val rdd2 = sc.parallelize(Seq((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w')))
val rdd3 = sc.parallelize(Array((1, 'a'), (3, 'b'), (4, 'c'), (4, 'd'))) val rdd3 = sc.parallelize(Seq((1, 'a'), (3, 'b'), (4, 'c'), (4, 'd')))
val rdd4 = sc.parallelize(Array((2, '@'))) val rdd4 = sc.parallelize(Seq((2, '@')))
val joined = rdd1.groupWith(rdd2, rdd3, rdd4).collect() val joined = rdd1.groupWith(rdd2, rdd3, rdd4).collect()
assert(joined.size === 4) assert(joined.size === 4)
val joinedSet = joined.map(x => (x._1, val joinedSet = joined.map(x => (x._1,
@ -480,7 +480,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("keys and values") { test("keys and values") {
val rdd = sc.parallelize(Array((1, "a"), (2, "b"))) val rdd = sc.parallelize(Seq((1, "a"), (2, "b")))
assert(rdd.keys.collect().toList === List(1, 2)) assert(rdd.keys.collect().toList === List(1, 2))
assert(rdd.values.collect().toList === List("a", "b")) assert(rdd.values.collect().toList === List("a", "b"))
} }
@ -517,9 +517,9 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
def getPartition(key: Any) = key.asInstanceOf[Int] def getPartition(key: Any) = key.asInstanceOf[Int]
} }
// partitionBy so we have a narrow dependency // partitionBy so we have a narrow dependency
val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p) val a = sc.parallelize(Seq((1, "a"), (2, "b"), (3, "c"))).partitionBy(p)
// more partitions/no partitioner so a shuffle dependency // more partitions/no partitioner so a shuffle dependency
val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) val b = sc.parallelize(Seq((2, "b"), (3, "cc"), (4, "d")), 4)
val c = a.subtract(b) val c = a.subtract(b)
assert(c.collect().toSet === Set((1, "a"), (3, "c"))) assert(c.collect().toSet === Set((1, "a"), (3, "c")))
// Ideally we could keep the original partitioner... // Ideally we could keep the original partitioner...
@ -527,8 +527,8 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("subtractByKey") { test("subtractByKey") {
val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2) val a = sc.parallelize(Seq((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2)
val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4) val b = sc.parallelize(Seq((2, 20), (3, 30), (4, 40)), 4)
val c = a.subtractByKey(b) val c = a.subtractByKey(b)
assert(c.collect().toSet === Set((1, "a"), (1, "a"))) assert(c.collect().toSet === Set((1, "a"), (1, "a")))
assert(c.partitions.size === a.partitions.size) assert(c.partitions.size === a.partitions.size)
@ -541,22 +541,22 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
def getPartition(key: Any) = key.asInstanceOf[Int] def getPartition(key: Any) = key.asInstanceOf[Int]
} }
// partitionBy so we have a narrow dependency // partitionBy so we have a narrow dependency
val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).partitionBy(p) val a = sc.parallelize(Seq((1, "a"), (1, "a"), (2, "b"), (3, "c"))).partitionBy(p)
// more partitions/no partitioner so a shuffle dependency // more partitions/no partitioner so a shuffle dependency
val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) val b = sc.parallelize(Seq((2, "b"), (3, "cc"), (4, "d")), 4)
val c = a.subtractByKey(b) val c = a.subtractByKey(b)
assert(c.collect().toSet === Set((1, "a"), (1, "a"))) assert(c.collect().toSet === Set((1, "a"), (1, "a")))
assert(c.partitioner.get === p) assert(c.partitioner.get === p)
} }
test("foldByKey") { test("foldByKey") {
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) val pairs = sc.parallelize(Seq((1, 1), (1, 2), (1, 3), (1, 1), (2, 1)))
val sums = pairs.foldByKey(0)(_ + _).collect() val sums = pairs.foldByKey(0)(_ + _).collect()
assert(sums.toSet === Set((1, 7), (2, 1))) assert(sums.toSet === Set((1, 7), (2, 1)))
} }
test("foldByKey with mutable result type") { test("foldByKey with mutable result type") {
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) val pairs = sc.parallelize(Seq((1, 1), (1, 2), (1, 3), (1, 1), (2, 1)))
val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache()
// Fold the values using in-place mutation // Fold the values using in-place mutation
val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect()
@ -571,7 +571,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("saveNewAPIHadoopFile should call setConf if format is configurable") { test("saveNewAPIHadoopFile should call setConf if format is configurable") {
val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(1)))) val pairs = sc.parallelize(Seq((Integer.valueOf(1), Integer.valueOf(1))))
// No error, non-configurable formats still work // No error, non-configurable formats still work
pairs.saveAsNewAPIHadoopFile[NewFakeFormat]("ignored") pairs.saveAsNewAPIHadoopFile[NewFakeFormat]("ignored")
@ -587,7 +587,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
test("The JobId on the driver and executors should be the same during the commit") { test("The JobId on the driver and executors should be the same during the commit") {
// Create more than one rdd to mimic stageId not equal to rddId // Create more than one rdd to mimic stageId not equal to rddId
val pairs = sc.parallelize(Array((1, 2), (2, 3)), 2) val pairs = sc.parallelize(Seq((1, 2), (2, 3)), 2)
.map { p => (Integer.valueOf(p._1 + 1), Integer.valueOf(p._2 + 1)) } .map { p => (Integer.valueOf(p._1 + 1), Integer.valueOf(p._2 + 1)) }
.filter { p => p._1 > 0 } .filter { p => p._1 > 0 }
pairs.saveAsNewAPIHadoopFile[YetAnotherFakeFormat]("ignored") pairs.saveAsNewAPIHadoopFile[YetAnotherFakeFormat]("ignored")
@ -595,7 +595,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("saveAsHadoopFile should respect configured output committers") { test("saveAsHadoopFile should respect configured output committers") {
val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(1)))) val pairs = sc.parallelize(Seq((Integer.valueOf(1), Integer.valueOf(1))))
val conf = new JobConf() val conf = new JobConf()
conf.setOutputCommitter(classOf[FakeOutputCommitter]) conf.setOutputCommitter(classOf[FakeOutputCommitter])
@ -607,7 +607,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("failure callbacks should be called before calling writer.close() in saveNewAPIHadoopFile") { test("failure callbacks should be called before calling writer.close() in saveNewAPIHadoopFile") {
val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(2))), 1) val pairs = sc.parallelize(Seq((Integer.valueOf(1), Integer.valueOf(2))), 1)
FakeWriterWithCallback.calledBy = "" FakeWriterWithCallback.calledBy = ""
FakeWriterWithCallback.exception = null FakeWriterWithCallback.exception = null
@ -622,7 +622,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("failure callbacks should be called before calling writer.close() in saveAsHadoopFile") { test("failure callbacks should be called before calling writer.close() in saveAsHadoopFile") {
val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(2))), 1) val pairs = sc.parallelize(Seq((Integer.valueOf(1), Integer.valueOf(2))), 1)
val conf = new JobConf() val conf = new JobConf()
FakeWriterWithCallback.calledBy = "" FakeWriterWithCallback.calledBy = ""
@ -640,7 +640,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
test("saveAsNewAPIHadoopDataset should support invalid output paths when " + test("saveAsNewAPIHadoopDataset should support invalid output paths when " +
"there are no files to be committed to an absolute output location") { "there are no files to be committed to an absolute output location") {
val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(2))), 1) val pairs = sc.parallelize(Seq((Integer.valueOf(1), Integer.valueOf(2))), 1)
def saveRddWithPath(path: String): Unit = { def saveRddWithPath(path: String): Unit = {
val job = NewJob.getInstance(new Configuration(sc.hadoopConfiguration)) val job = NewJob.getInstance(new Configuration(sc.hadoopConfiguration))
@ -668,7 +668,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
// for non-null invalid paths. // for non-null invalid paths.
test("saveAsHadoopDataset should respect empty output directory when " + test("saveAsHadoopDataset should respect empty output directory when " +
"there are no files to be committed to an absolute output location") { "there are no files to be committed to an absolute output location") {
val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(2))), 1) val pairs = sc.parallelize(Seq((Integer.valueOf(1), Integer.valueOf(2))), 1)
val conf = new JobConf() val conf = new JobConf()
conf.setOutputKeyClass(classOf[Integer]) conf.setOutputKeyClass(classOf[Integer])
@ -683,7 +683,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("lookup") { test("lookup") {
val pairs = sc.parallelize(Array((1, 2), (3, 4), (5, 6), (5, 7))) val pairs = sc.parallelize(Seq((1, 2), (3, 4), (5, 6), (5, 7)))
assert(pairs.partitioner === None) assert(pairs.partitioner === None)
assert(pairs.lookup(1) === Seq(2)) assert(pairs.lookup(1) === Seq(2))
@ -693,7 +693,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("lookup with partitioner") { test("lookup with partitioner") {
val pairs = sc.parallelize(Array((1, 2), (3, 4), (5, 6), (5, 7))) val pairs = sc.parallelize(Seq((1, 2), (3, 4), (5, 6), (5, 7)))
val p = new Partitioner { val p = new Partitioner {
def numPartitions: Int = 2 def numPartitions: Int = 2
@ -709,7 +709,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
} }
test("lookup with bad partitioner") { test("lookup with bad partitioner") {
val pairs = sc.parallelize(Array((1, 2), (3, 4), (5, 6), (5, 7))) val pairs = sc.parallelize(Seq((1, 2), (3, 4), (5, 6), (5, 7)))
val p = new Partitioner { val p = new Partitioner {
def numPartitions: Int = 2 def numPartitions: Int = 2

View file

@ -25,7 +25,7 @@ import org.apache.spark.internal.Logging
class SortingSuite extends SparkFunSuite with SharedSparkContext with Matchers with Logging { class SortingSuite extends SparkFunSuite with SharedSparkContext with Matchers with Logging {
test("sortByKey") { test("sortByKey") {
val pairs = sc.parallelize(Array((1, 0), (2, 0), (0, 0), (3, 0)), 2) val pairs = sc.parallelize(Seq((1, 0), (2, 0), (0, 0), (3, 0)), 2)
assert(pairs.sortByKey().collect() === Array((0, 0), (1, 0), (2, 0), (3, 0))) assert(pairs.sortByKey().collect() === Array((0, 0), (1, 0), (2, 0), (3, 0)))
} }

View file

@ -183,7 +183,7 @@ code constructs a graph from a collection of RDDs:
val sc: SparkContext val sc: SparkContext
// Create an RDD for the vertices // Create an RDD for the vertices
val users: RDD[(VertexId, (String, String))] = val users: RDD[(VertexId, (String, String))] =
sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")), sc.parallelize(Seq((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")),
(5L, ("franklin", "prof")), (2L, ("istoica", "prof")))) (5L, ("franklin", "prof")), (2L, ("istoica", "prof"))))
// Create an RDD for edges // Create an RDD for edges
val relationships: RDD[Edge[String]] = val relationships: RDD[Edge[String]] =
@ -420,7 +420,7 @@ interest or eliminate broken links. For example in the following code we remove
{% highlight scala %} {% highlight scala %}
// Create an RDD for the vertices // Create an RDD for the vertices
val users: RDD[(VertexId, (String, String))] = val users: RDD[(VertexId, (String, String))] =
sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")), sc.parallelize(Seq((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")),
(5L, ("franklin", "prof")), (2L, ("istoica", "prof")), (5L, ("franklin", "prof")), (2L, ("istoica", "prof")),
(4L, ("peter", "student")))) (4L, ("peter", "student"))))
// Create an RDD for edges // Create an RDD for edges

View file

@ -218,7 +218,7 @@ class GraphSuite extends SparkFunSuite with LocalSparkContext {
test("reverse with join elimination") { test("reverse with join elimination") {
withSpark { sc => withSpark { sc =>
val vertices: RDD[(VertexId, Int)] = sc.parallelize(Array((1L, 1), (2L, 2))) val vertices: RDD[(VertexId, Int)] = sc.parallelize(Seq((1L, 1), (2L, 2)))
val edges: RDD[Edge[Int]] = sc.parallelize(Array(Edge(1L, 2L, 0))) val edges: RDD[Edge[Int]] = sc.parallelize(Array(Edge(1L, 2L, 0)))
val graph = Graph(vertices, edges).reverse val graph = Graph(vertices, edges).reverse
val result = GraphXUtils.mapReduceTriplets[Int, Int, Int]( val result = GraphXUtils.mapReduceTriplets[Int, Int, Int](

View file

@ -101,7 +101,7 @@ class ConnectedComponentsSuite extends SparkFunSuite with LocalSparkContext {
withSpark { sc => withSpark { sc =>
// Create an RDD for the vertices // Create an RDD for the vertices
val users: RDD[(VertexId, (String, String))] = val users: RDD[(VertexId, (String, String))] =
sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")), sc.parallelize(Seq((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")),
(5L, ("franklin", "prof")), (2L, ("istoica", "prof")), (5L, ("franklin", "prof")), (2L, ("istoica", "prof")),
(4L, ("peter", "student")))) (4L, ("peter", "student"))))
// Create an RDD for edges // Create an RDD for edges

View file

@ -23,7 +23,7 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext
class MLPairRDDFunctionsSuite extends SparkFunSuite with MLlibTestSparkContext { class MLPairRDDFunctionsSuite extends SparkFunSuite with MLlibTestSparkContext {
test("topByKey") { test("topByKey") {
val topMap = sc.parallelize(Array((1, 7), (1, 3), (1, 6), (1, 1), (1, 2), (3, 2), (3, 7), (5, val topMap = sc.parallelize(Seq((1, 7), (1, 3), (1, 6), (1, 1), (1, 2), (3, 2), (3, 7), (5,
1), (3, 5)), 2) 1), (3, 5)), 2)
.topByKey(5) .topByKey(5)
.collectAsMap() .collectAsMap()

View file

@ -554,7 +554,7 @@ class DataFrameSuite extends QueryTest with SharedSparkSession {
} }
test("replace column using withColumns") { test("replace column using withColumns") {
val df2 = sparkContext.parallelize(Array((1, 2), (2, 3), (3, 4))).toDF("x", "y") val df2 = sparkContext.parallelize(Seq((1, 2), (2, 3), (3, 4))).toDF("x", "y")
val df3 = df2.withColumns(Seq("x", "newCol1", "newCol2"), val df3 = df2.withColumns(Seq("x", "newCol1", "newCol2"),
Seq(df2("x") + 1, df2("y"), df2("y") + 1)) Seq(df2("x") + 1, df2("y"), df2("y") + 1))
checkAnswer( checkAnswer(