-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-7237] Many user provided closures are not actually cleaned #5787
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from all commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
19e33b4
Add tests for all public RDD APIs that take in closures
9ac5f9b
Clean closures that are not currently cleaned
8ac3074
Prevent NPE in tests when CC is used outside of an app
e83699e
Clean one more
6498f44
Add missing test for groupBy
7a3cc80
Merge branch 'master' of github.com:apache/spark into clean-more
df3caa3
Address comments
7265865
Merge branch 'master' of github.com:apache/spark into clean-more
2f1f476
Merge branch 'master' of github.com:apache/spark into clean-more
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,10 +17,14 @@ | |
|
|
||
| package org.apache.spark.util | ||
|
|
||
| import java.io.NotSerializableException | ||
|
|
||
| import org.scalatest.FunSuite | ||
|
|
||
| import org.apache.spark.LocalSparkContext._ | ||
| import org.apache.spark.{SparkContext, SparkException} | ||
| import org.apache.spark.{TaskContext, SparkContext, SparkException} | ||
| import org.apache.spark.partial.CountEvaluator | ||
| import org.apache.spark.rdd.RDD | ||
|
|
||
| class ClosureCleanerSuite extends FunSuite { | ||
| test("closures inside an object") { | ||
|
|
@@ -52,17 +56,66 @@ class ClosureCleanerSuite extends FunSuite { | |
| } | ||
|
|
||
| test("toplevel return statements in closures are identified at cleaning time") { | ||
| val ex = intercept[SparkException] { | ||
| intercept[ReturnStatementInClosureException] { | ||
| TestObjectWithBogusReturns.run() | ||
| } | ||
|
|
||
| assert(ex.getMessage.contains("Return statements aren't allowed in Spark closures")) | ||
| } | ||
|
|
||
| test("return statements from named functions nested in closures don't raise exceptions") { | ||
| val result = TestObjectWithNestedReturns.run() | ||
| assert(result === 1) | ||
| } | ||
|
|
||
| test("user provided closures are actually cleaned") { | ||
|
|
||
| // We use return statements as an indication that a closure is actually being cleaned | ||
| // We expect closure cleaner to find the return statements in the user provided closures | ||
| def expectCorrectException(body: => Unit): Unit = { | ||
| try { | ||
| body | ||
| } catch { | ||
| case rse: ReturnStatementInClosureException => // Success! | ||
| case e @ (_: NotSerializableException | _: SparkException) => | ||
| fail(s"Expected ReturnStatementInClosureException, but got $e.\n" + | ||
| "This means the closure provided by user is not actually cleaned.") | ||
| } | ||
| } | ||
|
|
||
| withSpark(new SparkContext("local", "test")) { sc => | ||
| val rdd = sc.parallelize(1 to 10) | ||
| val pairRdd = rdd.map { i => (i, i) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testMap(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testFlatMap(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testFilter(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testSortBy(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testGroupBy(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testKeyBy(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testMapPartitions(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testMapPartitionsWithIndex(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testZipPartitions2(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testZipPartitions3(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testZipPartitions4(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testForeach(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testForeachPartition(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testReduce(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testTreeReduce(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testFold(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testAggregate(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testTreeAggregate(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testCombineByKey(pairRdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testAggregateByKey(pairRdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testFoldByKey(pairRdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testReduceByKey(pairRdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testMapValues(pairRdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testFlatMapValues(pairRdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testForeachAsync(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testForeachPartitionAsync(rdd) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testRunJob1(sc) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testRunJob2(sc) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testRunApproximateJob(sc) } | ||
| expectCorrectException { TestUserClosuresActuallyCleaned.testSubmitJob(sc) } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| // A non-serializable class we create in closures to make sure that we aren't | ||
|
|
@@ -187,3 +240,90 @@ class TestClassWithNesting(val y: Int) extends Serializable { | |
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Test whether closures passed in through public APIs are actually cleaned. | ||
| * | ||
| * We put a return statement in each of these closures as a mechanism to detect whether the | ||
| * ClosureCleaner actually cleaned our closure. If it did, then it would throw an appropriate | ||
| * exception explicitly complaining about the return statement. Otherwise, we know the | ||
| * ClosureCleaner did not actually clean our closure, in which case we should fail the test. | ||
| */ | ||
| private object TestUserClosuresActuallyCleaned { | ||
| def testMap(rdd: RDD[Int]): Unit = { rdd.map { _ => return; 0 }.count() } | ||
| def testFlatMap(rdd: RDD[Int]): Unit = { rdd.flatMap { _ => return; Seq() }.count() } | ||
| def testFilter(rdd: RDD[Int]): Unit = { rdd.filter { _ => return; true }.count() } | ||
| def testSortBy(rdd: RDD[Int]): Unit = { rdd.sortBy { _ => return; 1 }.count() } | ||
| def testKeyBy(rdd: RDD[Int]): Unit = { rdd.keyBy { _ => return; 1 }.count() } | ||
| def testGroupBy(rdd: RDD[Int]): Unit = { rdd.groupBy { _ => return; 1 }.count() } | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. No test for
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Oops good catch. I just verified that all other test methods are called. |
||
| def testMapPartitions(rdd: RDD[Int]): Unit = { rdd.mapPartitions { it => return; it }.count() } | ||
| def testMapPartitionsWithIndex(rdd: RDD[Int]): Unit = { | ||
| rdd.mapPartitionsWithIndex { (_, it) => return; it }.count() | ||
| } | ||
| def testZipPartitions2(rdd: RDD[Int]): Unit = { | ||
| rdd.zipPartitions(rdd) { case (it1, it2) => return; it1 }.count() | ||
| } | ||
| def testZipPartitions3(rdd: RDD[Int]): Unit = { | ||
| rdd.zipPartitions(rdd, rdd) { case (it1, it2, it3) => return; it1 }.count() | ||
| } | ||
| def testZipPartitions4(rdd: RDD[Int]): Unit = { | ||
| rdd.zipPartitions(rdd, rdd, rdd) { case (it1, it2, it3, it4) => return; it1 }.count() | ||
| } | ||
| def testForeach(rdd: RDD[Int]): Unit = { rdd.foreach { _ => return } } | ||
| def testForeachPartition(rdd: RDD[Int]): Unit = { rdd.foreachPartition { _ => return } } | ||
| def testReduce(rdd: RDD[Int]): Unit = { rdd.reduce { case (_, _) => return; 1 } } | ||
| def testTreeReduce(rdd: RDD[Int]): Unit = { rdd.treeReduce { case (_, _) => return; 1 } } | ||
| def testFold(rdd: RDD[Int]): Unit = { rdd.fold(0) { case (_, _) => return; 1 } } | ||
| def testAggregate(rdd: RDD[Int]): Unit = { | ||
| rdd.aggregate(0)({ case (_, _) => return; 1 }, { case (_, _) => return; 1 }) | ||
| } | ||
| def testTreeAggregate(rdd: RDD[Int]): Unit = { | ||
| rdd.treeAggregate(0)({ case (_, _) => return; 1 }, { case (_, _) => return; 1 }) | ||
| } | ||
|
|
||
| // Test pair RDD functions | ||
| def testCombineByKey(rdd: RDD[(Int, Int)]): Unit = { | ||
| rdd.combineByKey( | ||
| { _ => return; 1 }: Int => Int, | ||
| { case (_, _) => return; 1 }: (Int, Int) => Int, | ||
| { case (_, _) => return; 1 }: (Int, Int) => Int | ||
| ).count() | ||
| } | ||
| def testAggregateByKey(rdd: RDD[(Int, Int)]): Unit = { | ||
| rdd.aggregateByKey(0)({ case (_, _) => return; 1 }, { case (_, _) => return; 1 }).count() | ||
| } | ||
| def testFoldByKey(rdd: RDD[(Int, Int)]): Unit = { rdd.foldByKey(0) { case (_, _) => return; 1 } } | ||
| def testReduceByKey(rdd: RDD[(Int, Int)]): Unit = { rdd.reduceByKey { case (_, _) => return; 1 } } | ||
| def testMapValues(rdd: RDD[(Int, Int)]): Unit = { rdd.mapValues { _ => return; 1 } } | ||
| def testFlatMapValues(rdd: RDD[(Int, Int)]): Unit = { rdd.flatMapValues { _ => return; Seq() } } | ||
|
|
||
| // Test async RDD actions | ||
| def testForeachAsync(rdd: RDD[Int]): Unit = { rdd.foreachAsync { _ => return } } | ||
| def testForeachPartitionAsync(rdd: RDD[Int]): Unit = { rdd.foreachPartitionAsync { _ => return } } | ||
|
|
||
| // Test SparkContext runJob | ||
| def testRunJob1(sc: SparkContext): Unit = { | ||
| val rdd = sc.parallelize(1 to 10, 10) | ||
| sc.runJob(rdd, { (ctx: TaskContext, iter: Iterator[Int]) => return; 1 } ) | ||
| } | ||
| def testRunJob2(sc: SparkContext): Unit = { | ||
| val rdd = sc.parallelize(1 to 10, 10) | ||
| sc.runJob(rdd, { iter: Iterator[Int] => return; 1 } ) | ||
| } | ||
| def testRunApproximateJob(sc: SparkContext): Unit = { | ||
| val rdd = sc.parallelize(1 to 10, 10) | ||
| val evaluator = new CountEvaluator(1, 0.5) | ||
| sc.runApproximateJob( | ||
| rdd, { (ctx: TaskContext, iter: Iterator[Int]) => return; 1L }, evaluator, 1000) | ||
| } | ||
| def testSubmitJob(sc: SparkContext): Unit = { | ||
| val rdd = sc.parallelize(1 to 10, 10) | ||
| sc.submitJob( | ||
| rdd, | ||
| { _ => return; 1 }: Iterator[Int] => Int, | ||
| Seq.empty, | ||
| { case (_, _) => return }: (Int, Int) => Unit, | ||
| { return } | ||
| ) | ||
| } | ||
| } | ||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No sure if needs to clean
func. Sincefuncwill be passed tocombineByKey,combineByKeywill clean it.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this one's necessary because it's called inside another closure