From 795a763a39db9830d4f1917147b076ec46b6d238 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Mon, 20 Jan 2014 14:14:01 +0200 Subject: [PATCH] Change name to WriteInputFormatTestDataGenerator. Cleanup some var names. Use SPARK_HOME in path for writing test sequencefile data. --- ...> WriteInputFormatTestDataGenerator.scala} | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) rename core/src/main/scala/org/apache/spark/api/python/{WriteInputFormatTests.scala => WriteInputFormatTestDataGenerator.scala} (67%) diff --git a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTests.scala b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala similarity index 67% rename from core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTests.scala rename to core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala index 41e4612857d7e..163a0921c93cf 100644 --- a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTests.scala +++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala @@ -5,32 +5,33 @@ import org.apache.hadoop.io._ import scala.Array import java.io.{DataOutput, DataInput} -case class TestWritable(var str: String, var numi: Int, var numd: Double) extends Writable { +case class TestWritable(var str: String, var int: Int, var double: Double) extends Writable { def this() = this("", 0, 0.0) - def write(p1: DataOutput) = { - p1.writeUTF(str) - p1.writeInt(numi) - p1.writeDouble(numd) + def write(out: DataOutput) = { + out.writeUTF(str) + out.writeInt(int) + out.writeDouble(double) } - def readFields(p1: DataInput) = { - str = p1.readUTF() - numi = p1.readInt() - numd = p1.readDouble() + def readFields(in: DataInput) = { + str = in.readUTF() + int = in.readInt() + double = in.readDouble() } } -object WriteInputFormatTests extends App { +object WriteInputFormatTestDataGenerator extends App { import SparkContext._ val sc = new SparkContext("local[2]", "test") - val textPath = "../python/test_support/data/sftext/" - val intPath = "../python/test_support/data/sfint/" - val doublePath = "../python/test_support/data/sfdouble/" - val arrPath = "../python/test_support/data/sfarray/" - val classPath = "../python/test_support/data/sfclass/" + val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get + val textPath = s"$sparkHome/python/test_support/data/sftext/" + val intPath = s"$sparkHome/python/test_support/data/sfint/" + val doublePath = s"$sparkHome/python/test_support/data/sfdouble/" + val arrPath = s"$sparkHome/python/test_support/data/sfarray/" + val classPath = s"$sparkHome/python/test_support/data/sfclass/" val intKeys = Seq((1.0, "aa"), (2.0, "bb"), (2.0, "aa"), (3.0, "cc"), (2.0, "bb"), (1.0, "aa")) sc.parallelize(intKeys).saveAsSequenceFile(intPath) @@ -60,5 +61,4 @@ object WriteInputFormatTests extends App { classOf[Text], classOf[TestWritable], classOf[org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat[Text, TestWritable]]) - } \ No newline at end of file