-
Notifications
You must be signed in to change notification settings - Fork 29k
SPARK-897: preemptively serialize closures #143
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
Changes from all commits
ed2ccf0
5bfff24
abe816b
be1ecd6
b215dea
3b3f74a
64d04d2
bceab8a
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,90 @@ | ||
| /* | ||
| * 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.serializer; | ||
|
|
||
| import java.io.NotSerializableException | ||
|
|
||
| import org.scalatest.FunSuite | ||
|
|
||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.SparkException | ||
| import org.apache.spark.SharedSparkContext | ||
|
|
||
| /* A trivial (but unserializable) container for trivial functions */ | ||
| class UnserializableClass { | ||
| def op[T](x: T) = x.toString | ||
|
|
||
| def pred[T](x: T) = x.toString.length % 2 == 0 | ||
| } | ||
|
|
||
| class ProactiveClosureSerializationSuite extends FunSuite with SharedSparkContext { | ||
|
|
||
| def fixture = (sc.parallelize(0 until 1000).map(_.toString), new UnserializableClass) | ||
|
|
||
| test("throws expected serialization exceptions on actions") { | ||
| val (data, uc) = fixture | ||
|
|
||
| val ex = intercept[SparkException] { | ||
| data.map(uc.op(_)).count | ||
| } | ||
|
|
||
| assert(ex.getMessage.contains("Task not serializable")) | ||
| } | ||
|
|
||
| // There is probably a cleaner way to eliminate boilerplate here, but we're | ||
| // iterating over a map from transformation names to functions that perform that | ||
| // transformation on a given RDD, creating one test case for each | ||
|
|
||
| for (transformation <- | ||
| Map("map" -> xmap _, "flatMap" -> xflatMap _, "filter" -> xfilter _, | ||
| "mapWith" -> xmapWith _, "mapPartitions" -> xmapPartitions _, | ||
| "mapPartitionsWithIndex" -> xmapPartitionsWithIndex _, | ||
| "mapPartitionsWithContext" -> xmapPartitionsWithContext _, | ||
| "filterWith" -> xfilterWith _)) { | ||
| val (name, xf) = transformation | ||
|
|
||
| test(s"$name transformations throw proactive serialization exceptions") { | ||
| val (data, uc) = fixture | ||
|
|
||
| val ex = intercept[SparkException] { | ||
| xf(data, uc) | ||
| } | ||
|
|
||
| assert(ex.getMessage.contains("Task not serializable"), | ||
| s"RDD.$name doesn't proactively throw NotSerializableException") | ||
| } | ||
| } | ||
|
|
||
| private def xmap(x: RDD[String], uc: UnserializableClass): RDD[String] = | ||
| x.map(y=>uc.op(y)) | ||
| private def xmapWith(x: RDD[String], uc: UnserializableClass): RDD[String] = | ||
| x.mapWith(x => x.toString)((x,y)=>x + uc.op(y)) | ||
| private def xflatMap(x: RDD[String], uc: UnserializableClass): RDD[String] = | ||
| x.flatMap(y=>Seq(uc.op(y))) | ||
| private def xfilter(x: RDD[String], uc: UnserializableClass): RDD[String] = | ||
| x.filter(y=>uc.pred(y)) | ||
| private def xfilterWith(x: RDD[String], uc: UnserializableClass): RDD[String] = | ||
| x.filterWith(x => x.toString)((x,y)=>uc.pred(y)) | ||
| private def xmapPartitions(x: RDD[String], uc: UnserializableClass): RDD[String] = | ||
| x.mapPartitions(_.map(y=>uc.op(y))) | ||
| private def xmapPartitionsWithIndex(x: RDD[String], uc: UnserializableClass): RDD[String] = | ||
| x.mapPartitionsWithIndex((_, it) => it.map(y=>uc.op(y))) | ||
| private def xmapPartitionsWithContext(x: RDD[String], uc: UnserializableClass): RDD[String] = | ||
| x.mapPartitionsWithContext((_, it) => it.map(y=>uc.op(y))) | ||
|
|
||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -532,23 +532,32 @@ abstract class DStream[T: ClassTag] ( | |
| * 'this' DStream will be registered as an output stream and therefore materialized. | ||
| */ | ||
| def foreachRDD(foreachFunc: (RDD[T], Time) => Unit) { | ||
| new ForEachDStream(this, context.sparkContext.clean(foreachFunc)).register() | ||
| // because the DStream is reachable from the outer object here, and because | ||
| // DStreams can't be serialized with closures, we can't proactively check | ||
| // it for serializability and so we pass the optional false to SparkContext.clean | ||
| new ForEachDStream(this, context.sparkContext.clean(foreachFunc, false)).register() | ||
| } | ||
|
|
||
| /** | ||
| * Return a new DStream in which each RDD is generated by applying a function | ||
| * on each RDD of 'this' DStream. | ||
| */ | ||
| def transform[U: ClassTag](transformFunc: RDD[T] => RDD[U]): DStream[U] = { | ||
| transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r))) | ||
| // because the DStream is reachable from the outer object here, and because | ||
| // DStreams can't be serialized with closures, we can't proactively check | ||
| // it for serializability and so we pass the optional false to SparkContext.clean | ||
| transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r), false)) | ||
|
Contributor
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. Why do we need to set this to false for the tests? Is this because the tests don't have a serializer set up?
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. As far as I can tell, the issue here is that the
Contributor
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. document that inline (otherwise we will run into that again)
Contributor
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. @willb I think you missed this. Make sure you add comment above this line to explain the reason why we do not check serializable ...
Contributor
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. and for all other instances where that is set to false too |
||
| } | ||
|
|
||
| /** | ||
| * Return a new DStream in which each RDD is generated by applying a function | ||
| * on each RDD of 'this' DStream. | ||
| */ | ||
| def transform[U: ClassTag](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = { | ||
| val cleanedF = context.sparkContext.clean(transformFunc) | ||
| // because the DStream is reachable from the outer object here, and because | ||
| // DStreams can't be serialized with closures, we can't proactively check | ||
| // it for serializability and so we pass the optional false to SparkContext.clean | ||
| val cleanedF = context.sparkContext.clean(transformFunc, false) | ||
| val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => { | ||
| assert(rdds.length == 1) | ||
| cleanedF(rdds.head.asInstanceOf[RDD[T]], time) | ||
|
|
@@ -563,7 +572,10 @@ abstract class DStream[T: ClassTag] ( | |
| def transformWith[U: ClassTag, V: ClassTag]( | ||
| other: DStream[U], transformFunc: (RDD[T], RDD[U]) => RDD[V] | ||
| ): DStream[V] = { | ||
| val cleanedF = ssc.sparkContext.clean(transformFunc) | ||
| // because the DStream is reachable from the outer object here, and because | ||
| // DStreams can't be serialized with closures, we can't proactively check | ||
| // it for serializability and so we pass the optional false to SparkContext.clean | ||
| val cleanedF = ssc.sparkContext.clean(transformFunc, false) | ||
| transformWith(other, (rdd1: RDD[T], rdd2: RDD[U], time: Time) => cleanedF(rdd1, rdd2)) | ||
| } | ||
|
|
||
|
|
@@ -574,7 +586,10 @@ abstract class DStream[T: ClassTag] ( | |
| def transformWith[U: ClassTag, V: ClassTag]( | ||
| other: DStream[U], transformFunc: (RDD[T], RDD[U], Time) => RDD[V] | ||
| ): DStream[V] = { | ||
| val cleanedF = ssc.sparkContext.clean(transformFunc) | ||
| // because the DStream is reachable from the outer object here, and because | ||
| // DStreams can't be serialized with closures, we can't proactively check | ||
| // it for serializability and so we pass the optional false to SparkContext.clean | ||
| val cleanedF = ssc.sparkContext.clean(transformFunc, false) | ||
| val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => { | ||
| assert(rdds.length == 2) | ||
| val rdd1 = rdds(0).asInstanceOf[RDD[T]] | ||
|
|
||
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.
can you document checkSerializable in the doc? (like what exception does it throw)