Github user andrewor14 commented on a diff in the pull request:
https://github.com/apache/spark/pull/1499#discussion_r15621714
--- Diff:
core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala
---
@@ -0,0 +1,566 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util.collection
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.scalatest.FunSuite
+
+import org.apache.spark._
+import org.apache.spark.SparkContext._
+
+class ExternalSorterSuite extends FunSuite with LocalSparkContext {
+ test("empty data stream") {
+ val conf = new SparkConf(false)
+ conf.set("spark.shuffle.memoryFraction", "0.001")
+ conf.set("spark.shuffle.manager",
"org.apache.spark.shuffle.sort.SortShuffleManager")
+ sc = new SparkContext("local", "test", conf)
+
+ val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i,
j) => i + j)
+ val ord = implicitly[Ordering[Int]]
+
+ // Both aggregator and ordering
+ val sorter = new ExternalSorter[Int, Int, Int](
+ Some(agg), Some(new HashPartitioner(3)), Some(ord), None)
+ assert(sorter.iterator.toSeq === Seq())
+ sorter.stop()
+
+ // Only aggregator
+ val sorter2 = new ExternalSorter[Int, Int, Int](
+ Some(agg), Some(new HashPartitioner(3)), None, None)
+ assert(sorter2.iterator.toSeq === Seq())
+ sorter2.stop()
+
+ // Only ordering
+ val sorter3 = new ExternalSorter[Int, Int, Int](
+ None, Some(new HashPartitioner(3)), Some(ord), None)
+ assert(sorter3.iterator.toSeq === Seq())
+ sorter3.stop()
+
+ // Neither aggregator nor ordering
+ val sorter4 = new ExternalSorter[Int, Int, Int](
+ None, Some(new HashPartitioner(3)), None, None)
+ assert(sorter4.iterator.toSeq === Seq())
+ sorter4.stop()
+ }
+
+ test("few elements per partition") {
+ val conf = new SparkConf(false)
+ conf.set("spark.shuffle.memoryFraction", "0.001")
+ conf.set("spark.shuffle.manager",
"org.apache.spark.shuffle.sort.SortShuffleManager")
+ sc = new SparkContext("local", "test", conf)
+
+ val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i,
j) => i + j)
+ val ord = implicitly[Ordering[Int]]
+ val elements = Set((1, 1), (2, 2), (5, 5))
+ val expected = Set(
+ (0, Set()), (1, Set((1, 1))), (2, Set((2, 2))), (3, Set()), (4,
Set()),
+ (5, Set((5, 5))), (6, Set()))
+
+ // Both aggregator and ordering
+ val sorter = new ExternalSorter[Int, Int, Int](
+ Some(agg), Some(new HashPartitioner(7)), Some(ord), None)
+ sorter.write(elements.iterator)
+ assert(sorter.partitionedIterator.map(p => (p._1, p._2.toSet)).toSet
=== expected)
+ sorter.stop()
+
+ // Only aggregator
+ val sorter2 = new ExternalSorter[Int, Int, Int](
+ Some(agg), Some(new HashPartitioner(7)), None, None)
+ sorter2.write(elements.iterator)
+ assert(sorter2.partitionedIterator.map(p => (p._1, p._2.toSet)).toSet
=== expected)
+ sorter2.stop()
+
+ // Only ordering
+ val sorter3 = new ExternalSorter[Int, Int, Int](
+ None, Some(new HashPartitioner(7)), Some(ord), None)
+ sorter3.write(elements.iterator)
+ assert(sorter3.partitionedIterator.map(p => (p._1, p._2.toSet)).toSet
=== expected)
+ sorter3.stop()
+
+ // Neither aggregator nor ordering
+ val sorter4 = new ExternalSorter[Int, Int, Int](
+ None, Some(new HashPartitioner(7)), None, None)
+ sorter4.write(elements.iterator)
+ assert(sorter4.partitionedIterator.map(p => (p._1, p._2.toSet)).toSet
=== expected)
+ sorter4.stop()
+ }
+
+ test("empty partitions with spilling") {
+ val conf = new SparkConf(false)
+ conf.set("spark.shuffle.memoryFraction", "0.001")
+ conf.set("spark.shuffle.manager",
"org.apache.spark.shuffle.sort.SortShuffleManager")
+ sc = new SparkContext("local", "test", conf)
+
+ val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i,
j) => i + j)
+ val ord = implicitly[Ordering[Int]]
+ val elements = Iterator((1, 1), (5, 5)) ++ (0 until
100000).iterator.map(x => (2, 2))
+
+ val sorter = new ExternalSorter[Int, Int, Int](
+ None, Some(new HashPartitioner(7)), None, None)
+ sorter.write(elements)
+ assert(sc.env.blockManager.diskBlockManager.getAllFiles().length > 0)
// Make sure it spilled
+ val iter = sorter.partitionedIterator.map(p => (p._1, p._2.toList))
+ assert(iter.next() === (0, Nil))
+ assert(iter.next() === (1, List((1, 1))))
+ assert(iter.next() === (2, (0 until 100000).map(x => (2, 2)).toList))
+ assert(iter.next() === (3, Nil))
+ assert(iter.next() === (4, Nil))
+ assert(iter.next() === (5, List((5, 5))))
+ assert(iter.next() === (6, Nil))
+ sorter.stop()
+ }
+
+ test("spilling in local cluster") {
+ val conf = new SparkConf(true) // Load defaults, otherwise SPARK_HOME
is not found
+ conf.set("spark.shuffle.memoryFraction", "0.001")
+ conf.set("spark.shuffle.manager",
"org.apache.spark.shuffle.sort.SortShuffleManager")
+ sc = new SparkContext("local-cluster[1,1,512]", "test", conf)
+
+ // reduceByKey - should spill ~8 times
+ val rddA = sc.parallelize(0 until 100000).map(i => (i/2, i))
+ val resultA = rddA.reduceByKey(math.max).collect()
+ assert(resultA.length == 50000)
+ resultA.foreach { case(k, v) =>
+ if (v != k * 2 + 1) {
+ fail(s"Value for ${k} was wrong: expected ${k * 2 + 1}, got ${v}")
+ }
+ }
+
+ // groupByKey - should spill ~17 times
+ val rddB = sc.parallelize(0 until 100000).map(i => (i/4, i))
+ val resultB = rddB.groupByKey().collect()
+ assert(resultB.length == 25000)
+ resultB.foreach { case(i, seq) =>
+ val expected = Set(i * 4, i * 4 + 1, i * 4 + 2, i * 4 + 3)
+ if (seq.toSet != expected) {
+ fail(s"Value for ${i} was wrong: expected ${expected}, got
${seq.toSet}")
+ }
+ }
+
+ // cogroup - should spill ~7 times
+ val rddC1 = sc.parallelize(0 until 10000).map(i => (i, i))
+ val rddC2 = sc.parallelize(0 until 10000).map(i => (i%1000, i))
+ val resultC = rddC1.cogroup(rddC2).collect()
+ assert(resultC.length == 10000)
+ resultC.foreach { case(i, (seq1, seq2)) =>
+ i match {
+ case 0 =>
+ assert(seq1.toSet == Set[Int](0))
+ assert(seq2.toSet == Set[Int](0, 1000, 2000, 3000, 4000, 5000,
6000, 7000, 8000, 9000))
+ case 1 =>
+ assert(seq1.toSet == Set[Int](1))
+ assert(seq2.toSet == Set[Int](1, 1001, 2001, 3001, 4001, 5001,
6001, 7001, 8001, 9001))
+ case 5000 =>
+ assert(seq1.toSet == Set[Int](5000))
+ assert(seq2.toSet == Set[Int]())
+ case 9999 =>
+ assert(seq1.toSet == Set[Int](9999))
+ assert(seq2.toSet == Set[Int]())
+ case _ =>
+ }
+ }
+
+ // larger cogroup - should spill ~7 times
--- End diff --
strange that larger cogroup spills the same number of times than the
previous cogroup
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---