aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorPatrick Wendell <pwendell@gmail.com>2013-01-17 18:43:00 -0800
committerPatrick Wendell <pwendell@gmail.com>2013-01-17 18:43:00 -0800
commitee0314c3b3c60ab398e4ff1c802af7297fe27587 (patch)
tree6834b7c56431151a4c775a21dfbd9b36e5ef1dfd
parent70ba994d6d6f9e62269168e6a8a61ffce736a4d2 (diff)
parenteded21925ab549330d0337138fa1f81ae192e3e9 (diff)
downloadspark-ee0314c3b3c60ab398e4ff1c802af7297fe27587.tar.gz
spark-ee0314c3b3c60ab398e4ff1c802af7297fe27587.tar.bz2
spark-ee0314c3b3c60ab398e4ff1c802af7297fe27587.zip
Merge branch 'streaming' into streaming-java-api
-rw-r--r--core/src/main/scala/spark/rdd/CheckpointRDD.scala6
-rw-r--r--streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala6
2 files changed, 6 insertions, 6 deletions
diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala
index 86c63ca2f4..6f00f6ac73 100644
--- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala
@@ -80,12 +80,12 @@ private[spark] object CheckpointRDD extends Logging {
val serializer = SparkEnv.get.serializer.newInstance()
val serializeStream = serializer.serializeStream(fileOutputStream)
serializeStream.writeAll(iterator)
- fileOutputStream.close()
+ serializeStream.close()
if (!fs.rename(tempOutputPath, finalOutputPath)) {
if (!fs.delete(finalOutputPath, true)) {
throw new IOException("Checkpoint failed: failed to delete earlier output of task "
- + context.attemptId);
+ + context.attemptId)
}
if (!fs.rename(tempOutputPath, finalOutputPath)) {
throw new IOException("Checkpoint failed: failed to save output of task: "
@@ -119,7 +119,7 @@ private[spark] object CheckpointRDD extends Logging {
val rdd = sc.makeRDD(1 to 10, 10).flatMap(x => 1 to 10000)
val path = new Path(hdfsPath, "temp")
val fs = path.getFileSystem(new Configuration())
- sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, 10) _)
+ sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, 1024) _)
val cpRDD = new CheckpointRDD[Int](sc, path.toString)
assert(cpRDD.splits.length == rdd.splits.length, "Number of splits is not the same")
assert(cpRDD.collect.toList == rdd.collect.toList, "Data of splits not the same")
diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
index f9e03c607d..f73f9b1823 100644
--- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
@@ -151,10 +151,10 @@ class BasicOperationsSuite extends TestSuiteBase {
)
val updateStateOperation = (s: DStream[String]) => {
- val updateFunc = (values: Seq[Int], state: Option[RichInt]) => {
- Some(new RichInt(values.foldLeft(0)(_ + _) + state.map(_.self).getOrElse(0)))
+ val updateFunc = (values: Seq[Int], state: Option[Int]) => {
+ Some(values.foldLeft(0)(_ + _) + state.getOrElse(0))
}
- s.map(x => (x, 1)).updateStateByKey[RichInt](updateFunc).map(t => (t._1, t._2.self))
+ s.map(x => (x, 1)).updateStateByKey[Int](updateFunc)
}
testOperation(inputData, updateStateOperation, outputData, true)