diff --git a/.gitignore b/.gitignore index a31bf7e0091f4..7779980b74a22 100644 --- a/.gitignore +++ b/.gitignore @@ -1,9 +1,12 @@ *~ +*.#* +*#*# *.swp *.ipr *.iml *.iws .idea/ +.idea_modules/ sbt/*.jar .settings .cache @@ -16,6 +19,7 @@ third_party/libmesos.so third_party/libmesos.dylib conf/java-opts conf/*.sh +conf/*.cmd conf/*.properties conf/*.conf conf/*.xml diff --git a/.rat-excludes b/.rat-excludes index fb6323daf9211..9fc99d7fca35d 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -20,6 +20,7 @@ log4j.properties.template metrics.properties.template slaves spark-env.sh +spark-env.cmd spark-env.sh.template log4j-defaults.properties bootstrap-tooltip.js @@ -58,3 +59,4 @@ dist/* .*iws logs .*scalastyle-output.xml +.*dependency-reduced-pom.xml diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index c6b4aa5344757..b6c6b050fa331 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -8,5 +8,5 @@ submitting any copyrighted material via pull request, email, or other means you agree to license the material under the project's open source license and warrant that you have the legal authority to do so. -Please see [Contributing to Spark wiki page](https://cwiki.apache.org/SPARK/Contributing+to+Spark) +Please see the [Contributing to Spark wiki page](https://cwiki.apache.org/SPARK/Contributing+to+Spark) for more information. diff --git a/bin/spark-sql b/bin/spark-sql index ae096530cad04..9d66140b6aa17 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -24,7 +24,7 @@ set -o posix CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" -CLASS_NOT_FOUND_EXIT_STATUS=1 +CLASS_NOT_FOUND_EXIT_STATUS=101 # Figure out where Spark is installed FWDIR="$(cd "`dirname "$0"`"/..; pwd)" diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index f6703986bdf11..376e69cd997d5 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -116,7 +116,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) } } } else { - logWarning ("No need to commit output of task: " + taID.value) + logInfo ("No need to commit output of task: " + taID.value) } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 12b345a8fa7c3..f9ff4ea6ca157 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -775,17 +775,36 @@ private[spark] object PythonRDD extends Logging { }.toJavaRDD() } + private class AutoBatchedPickler(iter: Iterator[Any]) extends Iterator[Array[Byte]] { + private val pickle = new Pickler() + private var batch = 1 + private val buffer = new mutable.ArrayBuffer[Any] + + override def hasNext(): Boolean = iter.hasNext + + override def next(): Array[Byte] = { + while (iter.hasNext && buffer.length < batch) { + buffer += iter.next() + } + val bytes = pickle.dumps(buffer.toArray) + val size = bytes.length + // let 1M < size < 10M + if (size < 1024 * 1024) { + batch *= 2 + } else if (size > 1024 * 1024 * 10 && batch > 1) { + batch /= 2 + } + buffer.clear() + bytes + } + } + /** * Convert an RDD of Java objects to an RDD of serialized Python objects, that is usable by * PySpark. */ def javaToPython(jRDD: JavaRDD[Any]): JavaRDD[Array[Byte]] = { - jRDD.rdd.mapPartitions { iter => - val pickle = new Pickler - iter.map { row => - pickle.dumps(row) - } - } + jRDD.rdd.mapPartitions { iter => new AutoBatchedPickler(iter) } } /** diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index 6668797f5f8be..7903457b17e13 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -68,8 +68,8 @@ private[python] object SerDeUtil extends Logging { construct(args ++ Array("")) } else if (args.length == 2 && args(1).isInstanceOf[String]) { val typecode = args(0).asInstanceOf[String].charAt(0) - val data: String = args(1).asInstanceOf[String] - construct(typecode, machineCodes(typecode), data.getBytes("ISO-8859-1")) + val data: Array[Byte] = args(1).asInstanceOf[String].getBytes("ISO-8859-1") + construct(typecode, machineCodes(typecode), data) } else { super.construct(args) } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 5ed3575816a38..580a439c9a892 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -54,7 +54,7 @@ object SparkSubmit { private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" - private val CLASS_NOT_FOUND_EXIT_STATUS = 1 + private val CLASS_NOT_FOUND_EXIT_STATUS = 101 // Exposed for testing private[spark] var exitFn: () => Unit = () => System.exit(-1) @@ -172,7 +172,7 @@ object SparkSubmit { // All cluster managers OptionAssigner(args.master, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.master"), OptionAssigner(args.name, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.app.name"), - OptionAssigner(args.jars, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.jars"), + OptionAssigner(args.jars, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.jars"), OptionAssigner(args.driverMemory, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.driver.memory"), OptionAssigner(args.driverExtraClassPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, @@ -183,6 +183,7 @@ object SparkSubmit { sysProp = "spark.driver.extraLibraryPath"), // Standalone cluster only + OptionAssigner(args.jars, STANDALONE, CLUSTER, sysProp = "spark.jars"), OptionAssigner(args.driverMemory, STANDALONE, CLUSTER, clOption = "--memory"), OptionAssigner(args.driverCores, STANDALONE, CLUSTER, clOption = "--cores"), @@ -261,7 +262,7 @@ object SparkSubmit { } // In yarn-cluster mode, use yarn.Client as a wrapper around the user class - if (clusterManager == YARN && deployMode == CLUSTER) { + if (isYarnCluster) { childMainClass = "org.apache.spark.deploy.yarn.Client" if (args.primaryResource != SPARK_INTERNAL) { childArgs += ("--jar", args.primaryResource) @@ -279,7 +280,7 @@ object SparkSubmit { } // Read from default spark properties, if any - for ((k, v) <- args.getDefaultSparkProperties) { + for ((k, v) <- args.defaultSparkProperties) { sysProps.getOrElseUpdate(k, v) } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index d545f58c5da7e..92e0917743ed1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -57,12 +57,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { var pyFiles: String = null val sparkProperties: HashMap[String, String] = new HashMap[String, String]() - parseOpts(args.toList) - mergeSparkProperties() - checkRequiredArguments() - - /** Return default present in the currently defined defaults file. */ - def getDefaultSparkProperties = { + /** Default properties present in the currently defined defaults file. */ + lazy val defaultSparkProperties: HashMap[String, String] = { val defaultProperties = new HashMap[String, String]() if (verbose) SparkSubmit.printStream.println(s"Using properties file: $propertiesFile") Option(propertiesFile).foreach { filename => @@ -79,6 +75,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { defaultProperties } + parseOpts(args.toList) + mergeSparkProperties() + checkRequiredArguments() + /** * Fill in any undefined values based on the default properties file or options passed in through * the '--conf' flag. @@ -107,7 +107,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { } } - val properties = getDefaultSparkProperties + val properties = HashMap[String, String]() + properties.putAll(defaultSparkProperties) properties.putAll(sparkProperties) // Use properties file as fallback for values which have a direct analog to @@ -213,7 +214,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | verbose $verbose | |Default properties from $propertiesFile: - |${getDefaultSparkProperties.mkString(" ", "\n ", "\n")} + |${defaultSparkProperties.mkString(" ", "\n ", "\n")} """.stripMargin } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index c4ef8b63b0071..d25c29113d6da 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -67,6 +67,7 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { } private val appHeader = Seq( + "App ID", "App Name", "Started", "Completed", @@ -81,7 +82,8 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { val duration = UIUtils.formatDuration(info.endTime - info.startTime) val lastUpdated = UIUtils.formatDate(info.lastUpdated) - {info.name} + {info.id} + {info.name} {startTime} {endTime} {duration} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 2a3bd6ba0b9dc..432b552c58cd8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -489,23 +489,24 @@ private[spark] class Master( // First schedule drivers, they take strict precedence over applications // Randomization helps balance drivers val shuffledAliveWorkers = Random.shuffle(workers.toSeq.filter(_.state == WorkerState.ALIVE)) - val aliveWorkerNum = shuffledAliveWorkers.size + val numWorkersAlive = shuffledAliveWorkers.size var curPos = 0 + for (driver <- waitingDrivers.toList) { // iterate over a copy of waitingDrivers // We assign workers to each waiting driver in a round-robin fashion. For each driver, we // start from the last worker that was assigned a driver, and continue onwards until we have // explored all alive workers. - curPos = (curPos + 1) % aliveWorkerNum - val startPos = curPos var launched = false - while (curPos != startPos && !launched) { + var numWorkersVisited = 0 + while (numWorkersVisited < numWorkersAlive && !launched) { val worker = shuffledAliveWorkers(curPos) + numWorkersVisited += 1 if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { launchDriver(worker, driver) waitingDrivers -= driver launched = true } - curPos = (curPos + 1) % aliveWorkerNum + curPos = (curPos + 1) % numWorkersAlive } } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 99a88c13456df..3e49b6235aff3 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -137,7 +137,6 @@ class TaskMetrics extends Serializable { merged.localBlocksFetched += depMetrics.localBlocksFetched merged.remoteBlocksFetched += depMetrics.remoteBlocksFetched merged.remoteBytesRead += depMetrics.remoteBytesRead - merged.shuffleFinishTime = math.max(merged.shuffleFinishTime, depMetrics.shuffleFinishTime) } _shuffleReadMetrics = Some(merged) } @@ -177,11 +176,6 @@ case class InputMetrics(readMethod: DataReadMethod.Value) { */ @DeveloperApi class ShuffleReadMetrics extends Serializable { - /** - * Absolute time when this task finished reading shuffle data - */ - var shuffleFinishTime: Long = -1 - /** * Number of blocks fetched in this shuffle by this task (remote or local) */ diff --git a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala index 74074a8dcbfff..18172d359cb35 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala @@ -460,7 +460,7 @@ private[spark] class ReceivingConnection( if (currId != null) currId else super.getRemoteConnectionManagerId() } - // The reciever's remote address is the local socket on remote side : which is NOT + // The receiver's remote address is the local socket on remote side : which is NOT // the connection manager id of the receiver. // We infer that from the messages we receive on the receiver socket. private def processConnectionManagerId(header: MessageChunkHeader) { diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 09d3ea306515b..5aa7e94943561 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -501,7 +501,7 @@ private[nio] class ConnectionManager( def changeConnectionKeyInterest(connection: Connection, ops: Int) { keyInterestChangeRequests += ((connection.key, ops)) - // so that registerations happen ! + // so that registrations happen ! wakeupSelector() } @@ -832,7 +832,7 @@ private[nio] class ConnectionManager( } /** - * Send a message and block until an acknowldgment is received or an error occurs. + * Send a message and block until an acknowledgment is received or an error occurs. * @param connectionManagerId the message's destination * @param message the message being sent * @return a Future that either returns the acknowledgment message or captures an exception. diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index f6d9d12fe9006..51ba8c2d17834 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -872,7 +872,12 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) hadoopConf.set("mapred.output.compression.codec", c.getCanonicalName) hadoopConf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) } - hadoopConf.setOutputCommitter(classOf[FileOutputCommitter]) + + // Use configured output committer if already set + if (conf.getOutputCommitter == null) { + hadoopConf.setOutputCommitter(classOf[FileOutputCommitter]) + } + FileOutputFormat.setOutputPath(hadoopConf, SparkHadoopWriter.createPathFromString(path, hadoopConf)) saveAsHadoopDataset(hadoopConf) diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 4d6b5c81883b6..ceb434feb6ca1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -171,7 +171,6 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener } val shuffleReadMetrics = taskMetrics.shuffleReadMetrics match { case Some(metrics) => - " SHUFFLE_FINISH_TIME=" + metrics.shuffleFinishTime + " BLOCK_FETCHED_TOTAL=" + metrics.totalBlocksFetched + " BLOCK_FETCHED_LOCAL=" + metrics.localBlocksFetched + " BLOCK_FETCHED_REMOTE=" + metrics.remoteBlocksFetched + diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 2f45d192e1d4d..5c5ecc8434d78 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -34,7 +34,7 @@ private[spark] class SparkDeploySchedulerBackend( var client: AppClient = null var stopping = false var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ - var appId: String = _ + @volatile var appId: String = _ val registrationLock = new Object() var registrationDone = false diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index c4dddb2d1037e..6a48f673c4e78 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -255,7 +255,6 @@ private[spark] object JsonProtocol { } def shuffleReadMetricsToJson(shuffleReadMetrics: ShuffleReadMetrics): JValue = { - ("Shuffle Finish Time" -> shuffleReadMetrics.shuffleFinishTime) ~ ("Remote Blocks Fetched" -> shuffleReadMetrics.remoteBlocksFetched) ~ ("Local Blocks Fetched" -> shuffleReadMetrics.localBlocksFetched) ~ ("Fetch Wait Time" -> shuffleReadMetrics.fetchWaitTime) ~ @@ -590,7 +589,6 @@ private[spark] object JsonProtocol { def shuffleReadMetricsFromJson(json: JValue): ShuffleReadMetrics = { val metrics = new ShuffleReadMetrics - metrics.shuffleFinishTime = (json \ "Shuffle Finish Time").extract[Long] metrics.remoteBlocksFetched = (json \ "Remote Blocks Fetched").extract[Int] metrics.localBlocksFetched = (json \ "Local Blocks Fetched").extract[Int] metrics.fetchWaitTime = (json \ "Fetch Wait Time").extract[Long] diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index c76b7af18481d..ed063844323af 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1382,15 +1382,15 @@ private[spark] object Utils extends Logging { } /** - * Default number of retries in binding to a port. + * Default maximum number of retries when binding to a port before giving up. */ val portMaxRetries: Int = { if (sys.props.contains("spark.testing")) { // Set a higher number of retries for tests... - sys.props.get("spark.ports.maxRetries").map(_.toInt).getOrElse(100) + sys.props.get("spark.port.maxRetries").map(_.toInt).getOrElse(100) } else { Option(SparkEnv.get) - .flatMap(_.conf.getOption("spark.ports.maxRetries")) + .flatMap(_.conf.getOption("spark.port.maxRetries")) .map(_.toInt) .getOrElse(16) } diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 2a58c6a40d8e4..3f1cd0752e766 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -115,11 +115,13 @@ class JsonProtocolSuite extends FunSuite { workerInfo.lastHeartbeat = JsonConstants.currTimeInMillis workerInfo } + def createExecutorRunner(): ExecutorRunner = { new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host", new File("sparkHome"), new File("workDir"), "akka://worker", new SparkConf, ExecutorState.RUNNING) } + def createDriverRunner(): DriverRunner = { new DriverRunner(new SparkConf(), "driverId", new File("workDir"), new File("sparkHome"), createDriverDesc(), null, "akka://worker") diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 22b369a829418..0c324d8bdf6a4 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -154,6 +154,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { sysProps("spark.app.name") should be ("beauty") sysProps("spark.shuffle.spill") should be ("false") sysProps("SPARK_SUBMIT") should be ("true") + sysProps.keys should not contain ("spark.jars") } test("handles YARN client mode") { diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 63d3ddb4af98a..e84cc69592339 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -17,17 +17,21 @@ package org.apache.spark.rdd -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashSet +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.mapred._ +import org.apache.hadoop.util.Progressable + +import scala.collection.mutable.{ArrayBuffer, HashSet} import scala.util.Random -import org.scalatest.FunSuite import com.google.common.io.Files -import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.conf.{Configuration, Configurable} - -import org.apache.spark.SparkContext._ +import org.apache.hadoop.conf.{Configurable, Configuration} +import org.apache.hadoop.mapreduce.{JobContext => NewJobContext, OutputCommitter => NewOutputCommitter, +OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, +TaskAttemptContext => NewTaskAttempContext} import org.apache.spark.{Partitioner, SharedSparkContext} +import org.apache.spark.SparkContext._ +import org.scalatest.FunSuite class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { test("aggregateByKey") { @@ -467,7 +471,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val pairs = sc.parallelize(Array((new Integer(1), new Integer(1)))) // No error, non-configurable formats still work - pairs.saveAsNewAPIHadoopFile[FakeFormat]("ignored") + pairs.saveAsNewAPIHadoopFile[NewFakeFormat]("ignored") /* Check that configurable formats get configured: @@ -478,6 +482,17 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { pairs.saveAsNewAPIHadoopFile[ConfigTestFormat]("ignored") } + test("saveAsHadoopFile should respect configured output committers") { + val pairs = sc.parallelize(Array((new Integer(1), new Integer(1)))) + val conf = new JobConf() + conf.setOutputCommitter(classOf[FakeOutputCommitter]) + + FakeOutputCommitter.ran = false + pairs.saveAsHadoopFile("ignored", pairs.keyClass, pairs.valueClass, classOf[FakeOutputFormat], conf) + + assert(FakeOutputCommitter.ran, "OutputCommitter was never called") + } + test("lookup") { val pairs = sc.parallelize(Array((1,2), (3,4), (5,6), (5,7))) @@ -621,40 +636,86 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { and the test will therefore throw InstantiationException when saveAsNewAPIHadoopFile tries to instantiate them with Class.newInstance. */ + +/* + * Original Hadoop API + */ class FakeWriter extends RecordWriter[Integer, Integer] { + override def write(key: Integer, value: Integer): Unit = () - def close(p1: TaskAttemptContext) = () + override def close(reporter: Reporter): Unit = () +} + +class FakeOutputCommitter() extends OutputCommitter() { + override def setupJob(jobContext: JobContext): Unit = () + + override def needsTaskCommit(taskContext: TaskAttemptContext): Boolean = true + + override def setupTask(taskContext: TaskAttemptContext): Unit = () + + override def commitTask(taskContext: TaskAttemptContext): Unit = { + FakeOutputCommitter.ran = true + () + } + + override def abortTask(taskContext: TaskAttemptContext): Unit = () +} + +/* + * Used to communicate state between the test harness and the OutputCommitter. + */ +object FakeOutputCommitter { + var ran = false +} + +class FakeOutputFormat() extends OutputFormat[Integer, Integer]() { + override def getRecordWriter( + ignored: FileSystem, + job: JobConf, name: String, + progress: Progressable): RecordWriter[Integer, Integer] = { + new FakeWriter() + } + + override def checkOutputSpecs(ignored: FileSystem, job: JobConf): Unit = () +} + +/* + * New-style Hadoop API + */ +class NewFakeWriter extends NewRecordWriter[Integer, Integer] { + + def close(p1: NewTaskAttempContext) = () def write(p1: Integer, p2: Integer) = () } -class FakeCommitter extends OutputCommitter { - def setupJob(p1: JobContext) = () +class NewFakeCommitter extends NewOutputCommitter { + def setupJob(p1: NewJobContext) = () - def needsTaskCommit(p1: TaskAttemptContext): Boolean = false + def needsTaskCommit(p1: NewTaskAttempContext): Boolean = false - def setupTask(p1: TaskAttemptContext) = () + def setupTask(p1: NewTaskAttempContext) = () - def commitTask(p1: TaskAttemptContext) = () + def commitTask(p1: NewTaskAttempContext) = () - def abortTask(p1: TaskAttemptContext) = () + def abortTask(p1: NewTaskAttempContext) = () } -class FakeFormat() extends OutputFormat[Integer, Integer]() { +class NewFakeFormat() extends NewOutputFormat[Integer, Integer]() { - def checkOutputSpecs(p1: JobContext) = () + def checkOutputSpecs(p1: NewJobContext) = () - def getRecordWriter(p1: TaskAttemptContext): RecordWriter[Integer, Integer] = { - new FakeWriter() + def getRecordWriter(p1: NewTaskAttempContext): NewRecordWriter[Integer, Integer] = { + new NewFakeWriter() } - def getOutputCommitter(p1: TaskAttemptContext): OutputCommitter = { - new FakeCommitter() + def getOutputCommitter(p1: NewTaskAttempContext): NewOutputCommitter = { + new NewFakeCommitter() } } -class ConfigTestFormat() extends FakeFormat() with Configurable { +class ConfigTestFormat() extends NewFakeFormat() with Configurable { var setConfCalled = false def setConf(p1: Configuration) = { @@ -664,7 +725,7 @@ class ConfigTestFormat() extends FakeFormat() with Configurable { def getConf: Configuration = null - override def getRecordWriter(p1: TaskAttemptContext): RecordWriter[Integer, Integer] = { + override def getRecordWriter(p1: NewTaskAttempContext): NewRecordWriter[Integer, Integer] = { assert(setConfCalled, "setConf was never called") super.getRecordWriter(p1) } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 2b45d8b695853..f1f88c5fd3634 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -356,7 +356,6 @@ class JsonProtocolSuite extends FunSuite { } private def assertEquals(metrics1: ShuffleReadMetrics, metrics2: ShuffleReadMetrics) { - assert(metrics1.shuffleFinishTime === metrics2.shuffleFinishTime) assert(metrics1.remoteBlocksFetched === metrics2.remoteBlocksFetched) assert(metrics1.localBlocksFetched === metrics2.localBlocksFetched) assert(metrics1.fetchWaitTime === metrics2.fetchWaitTime) @@ -568,7 +567,6 @@ class JsonProtocolSuite extends FunSuite { t.inputMetrics = Some(inputMetrics) } else { val sr = new ShuffleReadMetrics - sr.shuffleFinishTime = b + c sr.remoteBytesRead = b + d sr.localBlocksFetched = e sr.fetchWaitTime = a + d @@ -806,7 +804,6 @@ class JsonProtocolSuite extends FunSuite { | "Memory Bytes Spilled": 800, | "Disk Bytes Spilled": 0, | "Shuffle Read Metrics": { - | "Shuffle Finish Time": 900, | "Remote Blocks Fetched": 800, | "Local Blocks Fetched": 700, | "Fetch Wait Time": 900, diff --git a/dev/run-tests b/dev/run-tests index 79401213a7fa2..c3d8f49cdd993 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -21,44 +21,73 @@ FWDIR="$(cd "`dirname $0`"/..; pwd)" cd "$FWDIR" -if [ -n "$AMPLAB_JENKINS_BUILD_PROFILE" ]; then - if [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop1.0" ]; then - export SBT_MAVEN_PROFILES_ARGS="-Dhadoop.version=1.0.4" - elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.0" ]; then - export SBT_MAVEN_PROFILES_ARGS="-Dhadoop.version=2.0.0-mr1-cdh4.1.1" - elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.2" ]; then - export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Dhadoop.version=2.2.0" - elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.3" ]; then - export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" +# Remove work directory +rm -rf ./work + +# Build against the right verison of Hadoop. +{ + if [ -n "$AMPLAB_JENKINS_BUILD_PROFILE" ]; then + if [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop1.0" ]; then + export SBT_MAVEN_PROFILES_ARGS="-Dhadoop.version=1.0.4" + elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.0" ]; then + export SBT_MAVEN_PROFILES_ARGS="-Dhadoop.version=2.0.0-mr1-cdh4.1.1" + elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.2" ]; then + export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Dhadoop.version=2.2.0" + elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.3" ]; then + export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" + fi fi -fi -if [ -z "$SBT_MAVEN_PROFILES_ARGS" ]; then - export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" -fi + if [ -z "$SBT_MAVEN_PROFILES_ARGS" ]; then + export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" + fi +} export SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Pkinesis-asl" -echo "SBT_MAVEN_PROFILES_ARGS=\"$SBT_MAVEN_PROFILES_ARGS\"" - -# Remove work directory -rm -rf ./work - -if test -x "$JAVA_HOME/bin/java"; then - declare java_cmd="$JAVA_HOME/bin/java" -else - declare java_cmd=java -fi -JAVA_VERSION=$($java_cmd -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') -[ "$JAVA_VERSION" -ge 18 ] && echo "" || echo "[Warn] Java 8 tests will not run because JDK version is < 1.8." +# Determine Java path and version. +{ + if test -x "$JAVA_HOME/bin/java"; then + declare java_cmd="$JAVA_HOME/bin/java" + else + declare java_cmd=java + fi + + # We can't use sed -r -e due to OS X / BSD compatibility; hence, all the parentheses. + JAVA_VERSION=$( + $java_cmd -version 2>&1 \ + | grep -e "^java version" --max-count=1 \ + | sed "s/java version \"\(.*\)\.\(.*\)\.\(.*\)\"/\1\2/" + ) + + if [ "$JAVA_VERSION" -lt 18 ]; then + echo "[warn] Java 8 tests will not run because JDK version is < 1.8." + fi +} -# Partial solution for SPARK-1455. Only run Hive tests if there are sql changes. +# Only run Hive tests if there are sql changes. +# Partial solution for SPARK-1455. if [ -n "$AMPLAB_JENKINS" ]; then git fetch origin master:master - diffs=`git diff --name-only master | grep "^\(sql/\)\|\(bin/spark-sql\)\|\(sbin/start-thriftserver.sh\)"` - if [ -n "$diffs" ]; then - echo "Detected changes in SQL. Will run Hive test suite." + + sql_diffs=$( + git diff --name-only master \ + | grep -e "^sql/" -e "^bin/spark-sql" -e "^sbin/start-thriftserver.sh" + ) + + non_sql_diffs=$( + git diff --name-only master \ + | grep -v -e "^sql/" -e "^bin/spark-sql" -e "^sbin/start-thriftserver.sh" + ) + + if [ -n "$sql_diffs" ]; then + echo "[info] Detected changes in SQL. Will run Hive test suite." _RUN_SQL_TESTS=true + + if [ -z "$non_sql_diffs" ]; then + echo "[info] Detected no changes except in SQL. Will only run SQL tests." + _SQL_TESTS_ONLY=true + fi fi fi @@ -70,42 +99,77 @@ echo "" echo "=========================================================================" echo "Running Apache RAT checks" echo "=========================================================================" -dev/check-license +./dev/check-license echo "" echo "=========================================================================" echo "Running Scala style checks" echo "=========================================================================" -dev/lint-scala +./dev/lint-scala echo "" echo "=========================================================================" echo "Running Python style checks" echo "=========================================================================" -dev/lint-python +./dev/lint-python + +echo "" +echo "=========================================================================" +echo "Building Spark" +echo "=========================================================================" + +{ + # We always build with Hive because the PySpark Spark SQL tests need it. + BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" + + echo "[info] Building Spark with these arguments: $BUILD_MVN_PROFILE_ARGS" + + # NOTE: echo "q" is needed because sbt on encountering a build file with failure + #+ (either resolution or compilation) prompts the user for input either q, r, etc + #+ to quit or retry. This echo is there to make it not block. + # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a + #+ single argument! + # QUESTION: Why doesn't 'yes "q"' work? + # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? + echo -e "q\n" \ + | sbt/sbt $BUILD_MVN_PROFILE_ARGS clean package assembly/assembly \ + | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" +} echo "" echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" -# Build Spark; we always build with Hive because the PySpark Spark SQL tests need it. -# echo "q" is needed because sbt on encountering a build file with failure -# (either resolution or compilation) prompts the user for input either q, r, -# etc to quit or retry. This echo is there to make it not block. -BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive " -echo -e "q\n" | sbt/sbt $BUILD_MVN_PROFILE_ARGS clean package assembly/assembly | \ - grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" - -# If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled: -if [ -n "$_RUN_SQL_TESTS" ]; then - SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" -fi -# echo "q" is needed because sbt on encountering a build file with failure -# (either resolution or compilation) prompts the user for input either q, r, -# etc to quit or retry. This echo is there to make it not block. -echo -e "q\n" | sbt/sbt $SBT_MAVEN_PROFILES_ARGS test | \ - grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" +{ + # If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled. + # This must be a single argument, as it is. + if [ -n "$_RUN_SQL_TESTS" ]; then + SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" + fi + + if [ -n "$_SQL_TESTS_ONLY" ]; then + # This must be an array of individual arguments. Otherwise, having one long string + #+ will be interpreted as a single test, which doesn't work. + SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "hive-thriftserver/test") + else + SBT_MAVEN_TEST_ARGS=("test") + fi + + echo "[info] Running Spark tests with these arguments: $SBT_MAVEN_PROFILES_ARGS ${SBT_MAVEN_TEST_ARGS[@]}" + + # NOTE: echo "q" is needed because sbt on encountering a build file with failure + #+ (either resolution or compilation) prompts the user for input either q, r, etc + #+ to quit or retry. This echo is there to make it not block. + # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a + #+ single argument! + #+ "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array. + # QUESTION: Why doesn't 'yes "q"' work? + # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? + echo -e "q\n" \ + | sbt/sbt $SBT_MAVEN_PROFILES_ARGS "${SBT_MAVEN_TEST_ARGS[@]}" \ + | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" +} echo "" echo "=========================================================================" @@ -117,4 +181,4 @@ echo "" echo "=========================================================================" echo "Detecting binary incompatibilites with MiMa" echo "=========================================================================" -dev/mima +./dev/mima diff --git a/docs/README.md b/docs/README.md index fdc89d2eb767a..79708c3df9106 100644 --- a/docs/README.md +++ b/docs/README.md @@ -20,12 +20,16 @@ In this directory you will find textfiles formatted using Markdown, with an ".md read those text files directly if you want. Start with index.md. The markdown code can be compiled to HTML using the [Jekyll tool](http://jekyllrb.com). -To use the `jekyll` command, you will need to have Jekyll installed. -The easiest way to do this is via a Ruby Gem, see the -[jekyll installation instructions](http://jekyllrb.com/docs/installation). -If not already installed, you need to install `kramdown` and `jekyll-redirect-from` Gems -with `sudo gem install kramdown jekyll-redirect-from`. -Execute `jekyll build` from the `docs/` directory. Compiling the site with Jekyll will create a directory +`Jekyll` and a few dependencies must be installed for this to work. We recommend +installing via the Ruby Gem dependency manager. Since the exact HTML output +varies between versions of Jekyll and its dependencies, we list specific versions here +in some cases: + + $ sudo gem install jekyll -v 1.4.3 + $ sudo gem uninstall kramdown -v 1.4.1 + $ sudo gem install jekyll-redirect-from + +Execute `jekyll` from the `docs/` directory. Compiling the site with Jekyll will create a directory called `_site` containing index.html as well as the rest of the compiled files. You can modify the default Jekyll build as follows: diff --git a/docs/_config.yml b/docs/_config.yml index d3ea2625c7448..7bc3a78e2d265 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -3,6 +3,11 @@ markdown: kramdown gems: - jekyll-redirect-from +# For some reason kramdown seems to behave differently on different +# OS/packages wrt encoding. So we hard code this config. +kramdown: + entity_output: numeric + # These allow the documentation to be updated with nerw releases # of Spark, Scala, and Mesos. SPARK_VERSION: 1.0.0-SNAPSHOT diff --git a/docs/configuration.md b/docs/configuration.md index 99faf51c6f3db..a6dd7245e1552 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -657,7 +657,7 @@ Apart from these, the following properties are also available, and may be useful spark.port.maxRetries 16 - Maximum number of retries when binding to a port before giving up. + Default maximum number of retries when binding to a port before giving up. diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 624cc744dfd51..01d378af574b5 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -286,7 +286,7 @@ We describe operations on distributed datasets later on. -One important parameter for parallel collections is the number of *slices* to cut the dataset into. Spark will run one task for each slice of the cluster. Typically you want 2-4 slices for each CPU in your cluster. Normally, Spark tries to set the number of slices automatically based on your cluster. However, you can also set it manually by passing it as a second parameter to `parallelize` (e.g. `sc.parallelize(data, 10)`). +One important parameter for parallel collections is the number of *partitions* to cut the dataset into. Spark will run one task for each partition of the cluster. Typically you want 2-4 partitions for each CPU in your cluster. Normally, Spark tries to set the number of partitions automatically based on your cluster. However, you can also set it manually by passing it as a second parameter to `parallelize` (e.g. `sc.parallelize(data, 10)`). Note: some places in the code use the term slices (a synonym for partitions) to maintain backward compatibility. ## External Datasets @@ -311,7 +311,7 @@ Some notes on reading files with Spark: * All of Spark's file-based input methods, including `textFile`, support running on directories, compressed files, and wildcards as well. For example, you can use `textFile("/my/directory")`, `textFile("/my/directory/*.txt")`, and `textFile("/my/directory/*.gz")`. -* The `textFile` method also takes an optional second argument for controlling the number of slices of the file. By default, Spark creates one slice for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of slices by passing a larger value. Note that you cannot have fewer slices than blocks. +* The `textFile` method also takes an optional second argument for controlling the number of partitions of the file. By default, Spark creates one partition for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of partitions by passing a larger value. Note that you cannot have fewer partitions than blocks. Apart from text files, Spark's Scala API also supports several other data formats: @@ -343,7 +343,7 @@ Some notes on reading files with Spark: * All of Spark's file-based input methods, including `textFile`, support running on directories, compressed files, and wildcards as well. For example, you can use `textFile("/my/directory")`, `textFile("/my/directory/*.txt")`, and `textFile("/my/directory/*.gz")`. -* The `textFile` method also takes an optional second argument for controlling the number of slices of the file. By default, Spark creates one slice for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of slices by passing a larger value. Note that you cannot have fewer slices than blocks. +* The `textFile` method also takes an optional second argument for controlling the number of partitions of the file. By default, Spark creates one partition for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of partitions by passing a larger value. Note that you cannot have fewer partitions than blocks. Apart from text files, Spark's Java API also supports several other data formats: @@ -375,7 +375,7 @@ Some notes on reading files with Spark: * All of Spark's file-based input methods, including `textFile`, support running on directories, compressed files, and wildcards as well. For example, you can use `textFile("/my/directory")`, `textFile("/my/directory/*.txt")`, and `textFile("/my/directory/*.gz")`. -* The `textFile` method also takes an optional second argument for controlling the number of slices of the file. By default, Spark creates one slice for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of slices by passing a larger value. Note that you cannot have fewer slices than blocks. +* The `textFile` method also takes an optional second argument for controlling the number of partitions of the file. By default, Spark creates one partition for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of partitions by passing a larger value. Note that you cannot have fewer partitions than blocks. Apart from text files, Spark's Python API also supports several other data formats: diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index c791c81f8bfd0..29b5491861bf3 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -248,8 +248,10 @@ You can also pass an option `--cores ` to control the number of cores The [`spark-submit` script](submitting-applications.html) provides the most straightforward way to submit a compiled Spark application to the cluster. For standalone clusters, Spark currently -only supports deploying the driver inside the client process that is submitting the application -(`client` deploy mode). +supports two deploy modes. In `client` mode, the driver is launched in the same process as the +client that submits the application. In `cluster` mode, however, the driver is launched from one +of the Worker processes inside the cluster, and the client process exits as soon as it fulfills +its responsibility of submitting the application without waiting for the application to finish. If your application is launched through Spark submit, then the application jar is automatically distributed to all worker nodes. For any additional jars that your application depends on, you @@ -307,7 +309,7 @@ tight firewall settings. For a complete list of ports to configure, see the By default, standalone scheduling clusters are resilient to Worker failures (insofar as Spark itself is resilient to losing work by moving it to other workers). However, the scheduler uses a Master to make scheduling decisions, and this (by default) creates a single point of failure: if the Master crashes, no new applications can be created. In order to circumvent this, we have two high availability schemes, detailed below. -# Standby Masters with ZooKeeper +## Standby Masters with ZooKeeper **Overview** @@ -347,7 +349,7 @@ There's an important distinction to be made between "registering with a Master" Due to this property, new Masters can be created at any time, and the only thing you need to worry about is that _new_ applications and Workers can find it to register with in case it becomes the leader. Once registered, you're taken care of. -# Single-Node Recovery with Local File System +## Single-Node Recovery with Local File System **Overview** diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index c498b41c43380..5212e19c41349 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -801,12 +801,12 @@ turning on some experimental options. ## Caching Data In Memory -Spark SQL can cache tables using an in-memory columnar format by calling `cacheTable("tableName")`. +Spark SQL can cache tables using an in-memory columnar format by calling `sqlContext.cacheTable("tableName")`. Then Spark SQL will scan only required columns and will automatically tune compression to minimize -memory usage and GC pressure. You can call `uncacheTable("tableName")` to remove the table from memory. +memory usage and GC pressure. You can call `sqlContext.uncacheTable("tableName")` to remove the table from memory. -Note that if you call `cache` rather than `cacheTable`, tables will _not_ be cached using -the in-memory columnar format, and therefore `cacheTable` is strongly recommended for this use case. +Note that if you call `schemaRDD.cache()` rather than `sqlContext.cacheTable(...)`, tables will _not_ be cached using +the in-memory columnar format, and therefore `sqlContext.cacheTable(...)` is strongly recommended for this use case. Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running `SET key=value` commands using SQL. diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 41f170580f452..5c21e912ea160 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -138,7 +138,7 @@ import org.apache.spark.streaming.api.java.*; import scala.Tuple2; // Create a local StreamingContext with two working thread and batch interval of 1 second -val conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount") +SparkConf conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount") JavaStreamingContext jssc = new JavaStreamingContext(conf, new Duration(1000)) {% endhighlight %} diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index abac71eaca595..fbeccd89b43b3 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -505,6 +505,7 @@ def tag_instance(instance, name): for i in range(0, 5): try: instance.add_tag(key='Name', value=name) + break except: print "Failed attempt %i of 5 to tag %s" % ((i + 1), name) if (i == 5): diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 5b1fa4d997eeb..70b6146e39a87 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -54,7 +54,7 @@ def update(i, vec, mat, ratings): if __name__ == "__main__": """ - Usage: als [M] [U] [F] [iterations] [slices]" + Usage: als [M] [U] [F] [iterations] [partitions]" """ print >> sys.stderr, """WARN: This is a naive implementation of ALS and is given as an @@ -66,10 +66,10 @@ def update(i, vec, mat, ratings): U = int(sys.argv[2]) if len(sys.argv) > 2 else 500 F = int(sys.argv[3]) if len(sys.argv) > 3 else 10 ITERATIONS = int(sys.argv[4]) if len(sys.argv) > 4 else 5 - slices = int(sys.argv[5]) if len(sys.argv) > 5 else 2 + partitions = int(sys.argv[5]) if len(sys.argv) > 5 else 2 - print "Running ALS with M=%d, U=%d, F=%d, iters=%d, slices=%d\n" % \ - (M, U, F, ITERATIONS, slices) + print "Running ALS with M=%d, U=%d, F=%d, iters=%d, partitions=%d\n" % \ + (M, U, F, ITERATIONS, partitions) R = matrix(rand(M, F)) * matrix(rand(U, F).T) ms = matrix(rand(M, F)) @@ -80,7 +80,7 @@ def update(i, vec, mat, ratings): usb = sc.broadcast(us) for i in range(ITERATIONS): - ms = sc.parallelize(range(M), slices) \ + ms = sc.parallelize(range(M), partitions) \ .map(lambda x: update(x, msb.value[x, :], usb.value, Rb.value)) \ .collect() # collect() returns a list, so array ends up being @@ -88,7 +88,7 @@ def update(i, vec, mat, ratings): ms = matrix(np.array(ms)[:, :, 0]) msb = sc.broadcast(ms) - us = sc.parallelize(range(U), slices) \ + us = sc.parallelize(range(U), partitions) \ .map(lambda x: update(x, usb.value[x, :], msb.value, Rb.value.T)) \ .collect() us = matrix(np.array(us)[:, :, 0]) diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index ee9036adfa281..a7c74e969cdb9 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -24,18 +24,18 @@ if __name__ == "__main__": """ - Usage: pi [slices] + Usage: pi [partitions] """ sc = SparkContext(appName="PythonPi") - slices = int(sys.argv[1]) if len(sys.argv) > 1 else 2 - n = 100000 * slices + partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 + n = 100000 * partitions def f(_): x = random() * 2 - 1 y = random() * 2 - 1 return 1 if x ** 2 + y ** 2 < 1 else 0 - count = sc.parallelize(xrange(1, n + 1), slices).map(f).reduce(add) + count = sc.parallelize(xrange(1, n + 1), partitions).map(f).reduce(add) print "Pi is roughly %f" % (4.0 * count / n) sc.stop() diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index bf331b542c438..00a281bfb6506 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -37,11 +37,11 @@ def generateGraph(): if __name__ == "__main__": """ - Usage: transitive_closure [slices] + Usage: transitive_closure [partitions] """ sc = SparkContext(appName="PythonTransitiveClosure") - slices = int(sys.argv[1]) if len(sys.argv) > 1 else 2 - tc = sc.parallelize(generateGraph(), slices).cache() + partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 + tc = sc.parallelize(generateGraph(), partitions).cache() # Linear transitive closure: each round grows paths by one edge, # by joining the graph's edges with the already-discovered paths. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index 04fbc9dbab8d1..2c8b245955d12 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -392,7 +392,7 @@ object VertexRDD { */ def apply[VD: ClassTag]( vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_, _], defaultVal: VD): VertexRDD[VD] = { - VertexRDD(vertices, edges, defaultVal, (a, b) => b) + VertexRDD(vertices, edges, defaultVal, (a, b) => a) } /** @@ -419,7 +419,7 @@ object VertexRDD { (vertexIter, routingTableIter) => val routingTable = if (routingTableIter.hasNext) routingTableIter.next() else RoutingTablePartition.empty - Iterator(ShippableVertexPartition(vertexIter, routingTable, defaultVal)) + Iterator(ShippableVertexPartition(vertexIter, routingTable, defaultVal, mergeFunc)) } new VertexRDD(vertexPartitions) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index dca54b8a7da86..5412d720475dc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -36,7 +36,7 @@ private[graphx] object ShippableVertexPartition { /** Construct a `ShippableVertexPartition` from the given vertices without any routing table. */ def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)]): ShippableVertexPartition[VD] = - apply(iter, RoutingTablePartition.empty, null.asInstanceOf[VD]) + apply(iter, RoutingTablePartition.empty, null.asInstanceOf[VD], (a, b) => a) /** * Construct a `ShippableVertexPartition` from the given vertices with the specified routing @@ -44,10 +44,28 @@ object ShippableVertexPartition { */ def apply[VD: ClassTag]( iter: Iterator[(VertexId, VD)], routingTable: RoutingTablePartition, defaultVal: VD) - : ShippableVertexPartition[VD] = { - val fullIter = iter ++ routingTable.iterator.map(vid => (vid, defaultVal)) - val (index, values, mask) = VertexPartitionBase.initFrom(fullIter, (a: VD, b: VD) => a) - new ShippableVertexPartition(index, values, mask, routingTable) + : ShippableVertexPartition[VD] = + apply(iter, routingTable, defaultVal, (a, b) => a) + + /** + * Construct a `ShippableVertexPartition` from the given vertices with the specified routing + * table, filling in missing vertices mentioned in the routing table using `defaultVal`, + * and merging duplicate vertex atrribute with mergeFunc. + */ + def apply[VD: ClassTag]( + iter: Iterator[(VertexId, VD)], routingTable: RoutingTablePartition, defaultVal: VD, + mergeFunc: (VD, VD) => VD): ShippableVertexPartition[VD] = { + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + // Merge the given vertices using mergeFunc + iter.foreach { pair => + map.setMerge(pair._1, pair._2, mergeFunc) + } + // Fill in missing vertices mentioned in the routing table + routingTable.iterator.foreach { vid => + map.changeValue(vid, defaultVal, identity) + } + + new ShippableVertexPartition(map.keySet, map._values, map.keySet.getBitSet, routingTable) } import scala.language.implicitConversions diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index cc86bafd2d644..42d3f21dbae98 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -99,4 +99,15 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } + test("mergeFunc") { + // test to see if the mergeFunc is working correctly + withSpark { sc => + val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3))) + val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) + val rdd = VertexRDD(verts, edges, 0, (a: Int, b: Int) => a + b) + // test merge function + assert(rdd.collect.toSet == Set((0L, 0), (1L, 3), (2L, 9))) + } + } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index fa0fa69f38634..9164c294ac7b8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -17,16 +17,20 @@ package org.apache.spark.mllib.api.python -import java.nio.{ByteBuffer, ByteOrder} +import java.io.OutputStream import scala.collection.JavaConverters._ +import scala.language.existentials +import scala.reflect.ClassTag + +import net.razorvine.pickle._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ import org.apache.spark.mllib.optimization._ -import org.apache.spark.mllib.linalg.{Matrix, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.random.{RandomRDDs => RG} import org.apache.spark.mllib.recommendation._ import org.apache.spark.mllib.regression._ @@ -40,11 +44,10 @@ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils + /** * :: DeveloperApi :: * The Java stubs necessary for the Python mllib bindings. - * - * See python/pyspark/mllib/_common.py for the mutually agreed upon data format. */ @DeveloperApi class PythonMLLibAPI extends Serializable { @@ -60,18 +63,17 @@ class PythonMLLibAPI extends Serializable { def loadLabeledPoints( jsc: JavaSparkContext, path: String, - minPartitions: Int): JavaRDD[Array[Byte]] = - MLUtils.loadLabeledPoints(jsc.sc, path, minPartitions).map(SerDe.serializeLabeledPoint) + minPartitions: Int): JavaRDD[LabeledPoint] = + MLUtils.loadLabeledPoints(jsc.sc, path, minPartitions) private def trainRegressionModel( trainFunc: (RDD[LabeledPoint], Vector) => GeneralizedLinearModel, - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], initialWeightsBA: Array[Byte]): java.util.LinkedList[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) - val initialWeights = SerDe.deserializeDoubleVector(initialWeightsBA) - val model = trainFunc(data, initialWeights) + val initialWeights = SerDe.loads(initialWeightsBA).asInstanceOf[Vector] + val model = trainFunc(data.rdd, initialWeights) val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(SerDe.serializeDoubleVector(model.weights)) + ret.add(SerDe.dumps(model.weights)) ret.add(model.intercept: java.lang.Double) ret } @@ -80,7 +82,7 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib LinearRegressionWithSGD.train() */ def trainLinearRegressionModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], numIterations: Int, stepSize: Double, miniBatchFraction: Double, @@ -106,7 +108,7 @@ class PythonMLLibAPI extends Serializable { trainRegressionModel( (data, initialWeights) => lrAlg.run(data, initialWeights), - dataBytesJRDD, + data, initialWeightsBA) } @@ -114,7 +116,7 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib LassoWithSGD.train() */ def trainLassoModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], numIterations: Int, stepSize: Double, regParam: Double, @@ -129,7 +131,7 @@ class PythonMLLibAPI extends Serializable { regParam, miniBatchFraction, initialWeights), - dataBytesJRDD, + data, initialWeightsBA) } @@ -137,7 +139,7 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib RidgeRegressionWithSGD.train() */ def trainRidgeModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], numIterations: Int, stepSize: Double, regParam: Double, @@ -152,7 +154,7 @@ class PythonMLLibAPI extends Serializable { regParam, miniBatchFraction, initialWeights), - dataBytesJRDD, + data, initialWeightsBA) } @@ -160,7 +162,7 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib SVMWithSGD.train() */ def trainSVMModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], numIterations: Int, stepSize: Double, regParam: Double, @@ -186,7 +188,7 @@ class PythonMLLibAPI extends Serializable { trainRegressionModel( (data, initialWeights) => SVMAlg.run(data, initialWeights), - dataBytesJRDD, + data, initialWeightsBA) } @@ -194,7 +196,7 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib LogisticRegressionWithSGD.train() */ def trainLogisticRegressionModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], numIterations: Int, stepSize: Double, miniBatchFraction: Double, @@ -220,7 +222,7 @@ class PythonMLLibAPI extends Serializable { trainRegressionModel( (data, initialWeights) => LogRegAlg.run(data, initialWeights), - dataBytesJRDD, + data, initialWeightsBA) } @@ -228,14 +230,13 @@ class PythonMLLibAPI extends Serializable { * Java stub for NaiveBayes.train() */ def trainNaiveBayes( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], lambda: Double): java.util.List[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) - val model = NaiveBayes.train(data, lambda) + val model = NaiveBayes.train(data.rdd, lambda) val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(SerDe.serializeDoubleVector(Vectors.dense(model.labels))) - ret.add(SerDe.serializeDoubleVector(Vectors.dense(model.pi))) - ret.add(SerDe.serializeDoubleMatrix(model.theta)) + ret.add(Vectors.dense(model.labels)) + ret.add(Vectors.dense(model.pi)) + ret.add(model.theta) ret } @@ -243,16 +244,12 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib KMeans.train() */ def trainKMeansModel( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[Vector], k: Int, maxIterations: Int, runs: Int, - initializationMode: String): java.util.List[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(bytes => SerDe.deserializeDoubleVector(bytes)) - val model = KMeans.train(data, k, maxIterations, runs, initializationMode) - val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(SerDe.serializeDoubleMatrix(model.clusterCenters.map(_.toArray))) - ret + initializationMode: String): KMeansModel = { + KMeans.train(data.rdd, k, maxIterations, runs, initializationMode) } /** @@ -262,13 +259,12 @@ class PythonMLLibAPI extends Serializable { * the Py4J documentation. */ def trainALSModel( - ratingsBytesJRDD: JavaRDD[Array[Byte]], + ratings: JavaRDD[Rating], rank: Int, iterations: Int, lambda: Double, blocks: Int): MatrixFactorizationModel = { - val ratings = ratingsBytesJRDD.rdd.map(SerDe.unpackRating) - ALS.train(ratings, rank, iterations, lambda, blocks) + ALS.train(ratings.rdd, rank, iterations, lambda, blocks) } /** @@ -278,14 +274,13 @@ class PythonMLLibAPI extends Serializable { * exit; see the Py4J documentation. */ def trainImplicitALSModel( - ratingsBytesJRDD: JavaRDD[Array[Byte]], + ratingsJRDD: JavaRDD[Rating], rank: Int, iterations: Int, lambda: Double, blocks: Int, alpha: Double): MatrixFactorizationModel = { - val ratings = ratingsBytesJRDD.rdd.map(SerDe.unpackRating) - ALS.trainImplicit(ratings, rank, iterations, lambda, blocks, alpha) + ALS.trainImplicit(ratingsJRDD.rdd, rank, iterations, lambda, blocks, alpha) } /** @@ -293,11 +288,11 @@ class PythonMLLibAPI extends Serializable { * This stub returns a handle to the Java object instead of the content of the Java object. * Extra care needs to be taken in the Python code to ensure it gets freed on exit; * see the Py4J documentation. - * @param dataBytesJRDD Training data + * @param data Training data * @param categoricalFeaturesInfoJMap Categorical features info, as Java map */ def trainDecisionTreeModel( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], algoStr: String, numClasses: Int, categoricalFeaturesInfoJMap: java.util.Map[Int, Int], @@ -307,8 +302,6 @@ class PythonMLLibAPI extends Serializable { minInstancesPerNode: Int, minInfoGain: Double): DecisionTreeModel = { - val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) - val algo = Algo.fromString(algoStr) val impurity = Impurities.fromString(impurityStr) @@ -322,44 +315,15 @@ class PythonMLLibAPI extends Serializable { minInstancesPerNode = minInstancesPerNode, minInfoGain = minInfoGain) - DecisionTree.train(data, strategy) - } - - /** - * Predict the label of the given data point. - * This is a Java stub for python DecisionTreeModel.predict() - * - * @param featuresBytes Serialized feature vector for data point - * @return predicted label - */ - def predictDecisionTreeModel( - model: DecisionTreeModel, - featuresBytes: Array[Byte]): Double = { - val features: Vector = SerDe.deserializeDoubleVector(featuresBytes) - model.predict(features) - } - - /** - * Predict the labels of the given data points. - * This is a Java stub for python DecisionTreeModel.predict() - * - * @param dataJRDD A JavaRDD with serialized feature vectors - * @return JavaRDD of serialized predictions - */ - def predictDecisionTreeModel( - model: DecisionTreeModel, - dataJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = { - val data = dataJRDD.rdd.map(xBytes => SerDe.deserializeDoubleVector(xBytes)) - model.predict(data).map(SerDe.serializeDouble) + DecisionTree.train(data.rdd, strategy) } /** * Java stub for mllib Statistics.colStats(X: RDD[Vector]). * TODO figure out return type. */ - def colStats(X: JavaRDD[Array[Byte]]): MultivariateStatisticalSummarySerialized = { - val cStats = Statistics.colStats(X.rdd.map(SerDe.deserializeDoubleVector(_))) - new MultivariateStatisticalSummarySerialized(cStats) + def colStats(rdd: JavaRDD[Vector]): MultivariateStatisticalSummary = { + Statistics.colStats(rdd.rdd) } /** @@ -367,19 +331,15 @@ class PythonMLLibAPI extends Serializable { * Returns the correlation matrix serialized into a byte array understood by deserializers in * pyspark. */ - def corr(X: JavaRDD[Array[Byte]], method: String): Array[Byte] = { - val inputMatrix = X.rdd.map(SerDe.deserializeDoubleVector(_)) - val result = Statistics.corr(inputMatrix, getCorrNameOrDefault(method)) - SerDe.serializeDoubleMatrix(SerDe.to2dArray(result)) + def corr(x: JavaRDD[Vector], method: String): Matrix = { + Statistics.corr(x.rdd, getCorrNameOrDefault(method)) } /** * Java stub for mllib Statistics.corr(x: RDD[Double], y: RDD[Double], method: String). */ - def corr(x: JavaRDD[Array[Byte]], y: JavaRDD[Array[Byte]], method: String): Double = { - val xDeser = x.rdd.map(SerDe.deserializeDouble(_)) - val yDeser = y.rdd.map(SerDe.deserializeDouble(_)) - Statistics.corr(xDeser, yDeser, getCorrNameOrDefault(method)) + def corr(x: JavaRDD[Double], y: JavaRDD[Double], method: String): Double = { + Statistics.corr(x.rdd, y.rdd, getCorrNameOrDefault(method)) } // used by the corr methods to retrieve the name of the correlation method passed in via pyspark @@ -411,10 +371,10 @@ class PythonMLLibAPI extends Serializable { def uniformRDD(jsc: JavaSparkContext, size: Long, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Double] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.uniformRDD(jsc.sc, size, parts, s).map(SerDe.serializeDouble) + RG.uniformRDD(jsc.sc, size, parts, s) } /** @@ -423,10 +383,10 @@ class PythonMLLibAPI extends Serializable { def normalRDD(jsc: JavaSparkContext, size: Long, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Double] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.normalRDD(jsc.sc, size, parts, s).map(SerDe.serializeDouble) + RG.normalRDD(jsc.sc, size, parts, s) } /** @@ -436,10 +396,10 @@ class PythonMLLibAPI extends Serializable { mean: Double, size: Long, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Double] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.poissonRDD(jsc.sc, mean, size, parts, s).map(SerDe.serializeDouble) + RG.poissonRDD(jsc.sc, mean, size, parts, s) } /** @@ -449,10 +409,10 @@ class PythonMLLibAPI extends Serializable { numRows: Long, numCols: Int, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Vector] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.uniformVectorRDD(jsc.sc, numRows, numCols, parts, s).map(SerDe.serializeDoubleVector) + RG.uniformVectorRDD(jsc.sc, numRows, numCols, parts, s) } /** @@ -462,10 +422,10 @@ class PythonMLLibAPI extends Serializable { numRows: Long, numCols: Int, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Vector] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.normalVectorRDD(jsc.sc, numRows, numCols, parts, s).map(SerDe.serializeDoubleVector) + RG.normalVectorRDD(jsc.sc, numRows, numCols, parts, s) } /** @@ -476,259 +436,168 @@ class PythonMLLibAPI extends Serializable { numRows: Long, numCols: Int, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Vector] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.poissonVectorRDD(jsc.sc, mean, numRows, numCols, parts, s).map(SerDe.serializeDoubleVector) + RG.poissonVectorRDD(jsc.sc, mean, numRows, numCols, parts, s) } } /** - * :: DeveloperApi :: - * MultivariateStatisticalSummary with Vector fields serialized. + * SerDe utility functions for PythonMLLibAPI. */ -@DeveloperApi -class MultivariateStatisticalSummarySerialized(val summary: MultivariateStatisticalSummary) - extends Serializable { +private[spark] object SerDe extends Serializable { - def mean: Array[Byte] = SerDe.serializeDoubleVector(summary.mean) + val PYSPARK_PACKAGE = "pyspark.mllib" - def variance: Array[Byte] = SerDe.serializeDoubleVector(summary.variance) + /** + * Base class used for pickle + */ + private[python] abstract class BasePickler[T: ClassTag] + extends IObjectPickler with IObjectConstructor { + + private val cls = implicitly[ClassTag[T]].runtimeClass + private val module = PYSPARK_PACKAGE + "." + cls.getName.split('.')(4) + private val name = cls.getSimpleName + + // register this to Pickler and Unpickler + def register(): Unit = { + Pickler.registerCustomPickler(this.getClass, this) + Pickler.registerCustomPickler(cls, this) + Unpickler.registerConstructor(module, name, this) + } - def count: Long = summary.count + def pickle(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + if (obj == this) { + out.write(Opcodes.GLOBAL) + out.write((module + "\n" + name + "\n").getBytes()) + } else { + pickler.save(this) // it will be memorized by Pickler + saveState(obj, out, pickler) + out.write(Opcodes.REDUCE) + } + } + + private[python] def saveObjects(out: OutputStream, pickler: Pickler, objects: Any*) = { + if (objects.length == 0 || objects.length > 3) { + out.write(Opcodes.MARK) + } + objects.foreach(pickler.save(_)) + val code = objects.length match { + case 1 => Opcodes.TUPLE1 + case 2 => Opcodes.TUPLE2 + case 3 => Opcodes.TUPLE3 + case _ => Opcodes.TUPLE + } + out.write(code) + } - def numNonzeros: Array[Byte] = SerDe.serializeDoubleVector(summary.numNonzeros) + private[python] def saveState(obj: Object, out: OutputStream, pickler: Pickler) + } - def max: Array[Byte] = SerDe.serializeDoubleVector(summary.max) + // Pickler for DenseVector + private[python] class DenseVectorPickler extends BasePickler[DenseVector] { - def min: Array[Byte] = SerDe.serializeDoubleVector(summary.min) -} + def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + val vector: DenseVector = obj.asInstanceOf[DenseVector] + saveObjects(out, pickler, vector.toArray) + } -/** - * SerDe utility functions for PythonMLLibAPI. - */ -private[spark] object SerDe extends Serializable { - private val DENSE_VECTOR_MAGIC: Byte = 1 - private val SPARSE_VECTOR_MAGIC: Byte = 2 - private val DENSE_MATRIX_MAGIC: Byte = 3 - private val LABELED_POINT_MAGIC: Byte = 4 - - private[python] def deserializeDoubleVector(bytes: Array[Byte], offset: Int = 0): Vector = { - require(bytes.length - offset >= 5, "Byte array too short") - val magic = bytes(offset) - if (magic == DENSE_VECTOR_MAGIC) { - deserializeDenseVector(bytes, offset) - } else if (magic == SPARSE_VECTOR_MAGIC) { - deserializeSparseVector(bytes, offset) - } else { - throw new IllegalArgumentException("Magic " + magic + " is wrong.") + def construct(args: Array[Object]): Object = { + require(args.length == 1) + if (args.length != 1) { + throw new PickleException("should be 1") + } + new DenseVector(args(0).asInstanceOf[Array[Double]]) } } - private[python] def deserializeDouble(bytes: Array[Byte], offset: Int = 0): Double = { - require(bytes.length - offset == 8, "Wrong size byte array for Double") - val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) - bb.order(ByteOrder.nativeOrder()) - bb.getDouble - } - - private[python] def deserializeDenseVector(bytes: Array[Byte], offset: Int = 0): Vector = { - val packetLength = bytes.length - offset - require(packetLength >= 5, "Byte array too short") - val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) - bb.order(ByteOrder.nativeOrder()) - val magic = bb.get() - require(magic == DENSE_VECTOR_MAGIC, "Invalid magic: " + magic) - val length = bb.getInt() - require (packetLength == 5 + 8 * length, "Invalid packet length: " + packetLength) - val db = bb.asDoubleBuffer() - val ans = new Array[Double](length.toInt) - db.get(ans) - Vectors.dense(ans) - } - - private[python] def deserializeSparseVector(bytes: Array[Byte], offset: Int = 0): Vector = { - val packetLength = bytes.length - offset - require(packetLength >= 9, "Byte array too short") - val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) - bb.order(ByteOrder.nativeOrder()) - val magic = bb.get() - require(magic == SPARSE_VECTOR_MAGIC, "Invalid magic: " + magic) - val size = bb.getInt() - val nonZeros = bb.getInt() - require (packetLength == 9 + 12 * nonZeros, "Invalid packet length: " + packetLength) - val ib = bb.asIntBuffer() - val indices = new Array[Int](nonZeros) - ib.get(indices) - bb.position(bb.position() + 4 * nonZeros) - val db = bb.asDoubleBuffer() - val values = new Array[Double](nonZeros) - db.get(values) - Vectors.sparse(size, indices, values) - } + // Pickler for DenseMatrix + private[python] class DenseMatrixPickler extends BasePickler[DenseMatrix] { - /** - * Returns an 8-byte array for the input Double. - * - * Note: we currently do not use a magic byte for double for storage efficiency. - * This should be reconsidered when we add Ser/De for other 8-byte types (e.g. Long), for safety. - * The corresponding deserializer, deserializeDouble, needs to be modified as well if the - * serialization scheme changes. - */ - private[python] def serializeDouble(double: Double): Array[Byte] = { - val bytes = new Array[Byte](8) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.putDouble(double) - bytes - } - - private[python] def serializeDenseVector(doubles: Array[Double]): Array[Byte] = { - val len = doubles.length - val bytes = new Array[Byte](5 + 8 * len) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.put(DENSE_VECTOR_MAGIC) - bb.putInt(len) - val db = bb.asDoubleBuffer() - db.put(doubles) - bytes - } - - private[python] def serializeSparseVector(vector: SparseVector): Array[Byte] = { - val nonZeros = vector.indices.length - val bytes = new Array[Byte](9 + 12 * nonZeros) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.put(SPARSE_VECTOR_MAGIC) - bb.putInt(vector.size) - bb.putInt(nonZeros) - val ib = bb.asIntBuffer() - ib.put(vector.indices) - bb.position(bb.position() + 4 * nonZeros) - val db = bb.asDoubleBuffer() - db.put(vector.values) - bytes - } - - private[python] def serializeDoubleVector(vector: Vector): Array[Byte] = vector match { - case s: SparseVector => - serializeSparseVector(s) - case _ => - serializeDenseVector(vector.toArray) - } - - private[python] def deserializeDoubleMatrix(bytes: Array[Byte]): Array[Array[Double]] = { - val packetLength = bytes.length - if (packetLength < 9) { - throw new IllegalArgumentException("Byte array too short.") + def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + val m: DenseMatrix = obj.asInstanceOf[DenseMatrix] + saveObjects(out, pickler, m.numRows, m.numCols, m.values) } - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - val magic = bb.get() - if (magic != DENSE_MATRIX_MAGIC) { - throw new IllegalArgumentException("Magic " + magic + " is wrong.") + + def construct(args: Array[Object]): Object = { + if (args.length != 3) { + throw new PickleException("should be 3") + } + new DenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], + args(2).asInstanceOf[Array[Double]]) } - val rows = bb.getInt() - val cols = bb.getInt() - if (packetLength != 9 + 8 * rows * cols) { - throw new IllegalArgumentException("Size " + rows + "x" + cols + " is wrong.") + } + + // Pickler for SparseVector + private[python] class SparseVectorPickler extends BasePickler[SparseVector] { + + def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + val v: SparseVector = obj.asInstanceOf[SparseVector] + saveObjects(out, pickler, v.size, v.indices, v.values) } - val db = bb.asDoubleBuffer() - val ans = new Array[Array[Double]](rows.toInt) - for (i <- 0 until rows.toInt) { - ans(i) = new Array[Double](cols.toInt) - db.get(ans(i)) + + def construct(args: Array[Object]): Object = { + if (args.length != 3) { + throw new PickleException("should be 3") + } + new SparseVector(args(0).asInstanceOf[Int], args(1).asInstanceOf[Array[Int]], + args(2).asInstanceOf[Array[Double]]) } - ans } - private[python] def serializeDoubleMatrix(doubles: Array[Array[Double]]): Array[Byte] = { - val rows = doubles.length - var cols = 0 - if (rows > 0) { - cols = doubles(0).length + // Pickler for LabeledPoint + private[python] class LabeledPointPickler extends BasePickler[LabeledPoint] { + + def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + val point: LabeledPoint = obj.asInstanceOf[LabeledPoint] + saveObjects(out, pickler, point.label, point.features) } - val bytes = new Array[Byte](9 + 8 * rows * cols) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.put(DENSE_MATRIX_MAGIC) - bb.putInt(rows) - bb.putInt(cols) - val db = bb.asDoubleBuffer() - for (i <- 0 until rows) { - db.put(doubles(i)) + + def construct(args: Array[Object]): Object = { + if (args.length != 2) { + throw new PickleException("should be 2") + } + new LabeledPoint(args(0).asInstanceOf[Double], args(1).asInstanceOf[Vector]) } - bytes } - private[python] def serializeLabeledPoint(p: LabeledPoint): Array[Byte] = { - val fb = serializeDoubleVector(p.features) - val bytes = new Array[Byte](1 + 8 + fb.length) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.put(LABELED_POINT_MAGIC) - bb.putDouble(p.label) - bb.put(fb) - bytes - } + // Pickler for Rating + private[python] class RatingPickler extends BasePickler[Rating] { - private[python] def deserializeLabeledPoint(bytes: Array[Byte]): LabeledPoint = { - require(bytes.length >= 9, "Byte array too short") - val magic = bytes(0) - if (magic != LABELED_POINT_MAGIC) { - throw new IllegalArgumentException("Magic " + magic + " is wrong.") + def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + val rating: Rating = obj.asInstanceOf[Rating] + saveObjects(out, pickler, rating.user, rating.product, rating.rating) } - val labelBytes = ByteBuffer.wrap(bytes, 1, 8) - labelBytes.order(ByteOrder.nativeOrder()) - val label = labelBytes.asDoubleBuffer().get(0) - LabeledPoint(label, deserializeDoubleVector(bytes, 9)) - } - // Reformat a Matrix into Array[Array[Double]] for serialization - private[python] def to2dArray(matrix: Matrix): Array[Array[Double]] = { - val values = matrix.toArray - Array.tabulate(matrix.numRows, matrix.numCols)((i, j) => values(i + j * matrix.numRows)) + def construct(args: Array[Object]): Object = { + if (args.length != 3) { + throw new PickleException("should be 3") + } + new Rating(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], + args(2).asInstanceOf[Double]) + } } + def initialize(): Unit = { + new DenseVectorPickler().register() + new DenseMatrixPickler().register() + new SparseVectorPickler().register() + new LabeledPointPickler().register() + new RatingPickler().register() + } - /** Unpack a Rating object from an array of bytes */ - private[python] def unpackRating(ratingBytes: Array[Byte]): Rating = { - val bb = ByteBuffer.wrap(ratingBytes) - bb.order(ByteOrder.nativeOrder()) - val user = bb.getInt() - val product = bb.getInt() - val rating = bb.getDouble() - new Rating(user, product, rating) + def dumps(obj: AnyRef): Array[Byte] = { + new Pickler().dumps(obj) } - /** Unpack a tuple of Ints from an array of bytes */ - def unpackTuple(tupleBytes: Array[Byte]): (Int, Int) = { - val bb = ByteBuffer.wrap(tupleBytes) - bb.order(ByteOrder.nativeOrder()) - val v1 = bb.getInt() - val v2 = bb.getInt() - (v1, v2) + def loads(bytes: Array[Byte]): AnyRef = { + new Unpickler().loads(bytes) } - /** - * Serialize a Rating object into an array of bytes. - * It can be deserialized using RatingDeserializer(). - * - * @param rate the Rating object to serialize - * @return - */ - def serializeRating(rate: Rating): Array[Byte] = { - val len = 3 - val bytes = new Array[Byte](4 + 8 * len) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.putInt(len) - val db = bb.asDoubleBuffer() - db.put(rate.user.toDouble) - db.put(rate.product.toDouble) - db.put(rate.rating) - bytes + /* convert object into Tuple */ + def asTupleRDD(rdd: RDD[Array[Any]]): RDD[(Int, Int)] = { + rdd.map(x => (x(0).asInstanceOf[Int], x(1).asInstanceOf[Int])) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala index 70e23033c8754..54ee930d61003 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala @@ -18,13 +18,17 @@ package org.apache.spark.mllib.linalg import com.github.fommil.netlib.{BLAS => NetlibBLAS, F2jBLAS} +import com.github.fommil.netlib.BLAS.{getInstance => NativeBLAS} + +import org.apache.spark.Logging /** * BLAS routines for MLlib's vectors and matrices. */ -private[mllib] object BLAS extends Serializable { +private[mllib] object BLAS extends Serializable with Logging { @transient private var _f2jBLAS: NetlibBLAS = _ + @transient private var _nativeBLAS: NetlibBLAS = _ // For level-1 routines, we use Java implementation. private def f2jBLAS: NetlibBLAS = { @@ -197,4 +201,328 @@ private[mllib] object BLAS extends Serializable { throw new IllegalArgumentException(s"scal doesn't support vector type ${x.getClass}.") } } + + // For level-3 routines, we use the native BLAS. + private def nativeBLAS: NetlibBLAS = { + if (_nativeBLAS == null) { + _nativeBLAS = NativeBLAS + } + _nativeBLAS + } + + /** + * C := alpha * A * B + beta * C + * @param transA whether to use the transpose of matrix A (true), or A itself (false). + * @param transB whether to use the transpose of matrix B (true), or B itself (false). + * @param alpha a scalar to scale the multiplication A * B. + * @param A the matrix A that will be left multiplied to B. Size of m x k. + * @param B the matrix B that will be left multiplied by A. Size of k x n. + * @param beta a scalar that can be used to scale matrix C. + * @param C the resulting matrix C. Size of m x n. + */ + def gemm( + transA: Boolean, + transB: Boolean, + alpha: Double, + A: Matrix, + B: DenseMatrix, + beta: Double, + C: DenseMatrix): Unit = { + if (alpha == 0.0) { + logDebug("gemm: alpha is equal to 0. Returning C.") + } else { + A match { + case sparse: SparseMatrix => + gemm(transA, transB, alpha, sparse, B, beta, C) + case dense: DenseMatrix => + gemm(transA, transB, alpha, dense, B, beta, C) + case _ => + throw new IllegalArgumentException(s"gemm doesn't support matrix type ${A.getClass}.") + } + } + } + + /** + * C := alpha * A * B + beta * C + * + * @param alpha a scalar to scale the multiplication A * B. + * @param A the matrix A that will be left multiplied to B. Size of m x k. + * @param B the matrix B that will be left multiplied by A. Size of k x n. + * @param beta a scalar that can be used to scale matrix C. + * @param C the resulting matrix C. Size of m x n. + */ + def gemm( + alpha: Double, + A: Matrix, + B: DenseMatrix, + beta: Double, + C: DenseMatrix): Unit = { + gemm(false, false, alpha, A, B, beta, C) + } + + /** + * C := alpha * A * B + beta * C + * For `DenseMatrix` A. + */ + private def gemm( + transA: Boolean, + transB: Boolean, + alpha: Double, + A: DenseMatrix, + B: DenseMatrix, + beta: Double, + C: DenseMatrix): Unit = { + val mA: Int = if (!transA) A.numRows else A.numCols + val nB: Int = if (!transB) B.numCols else B.numRows + val kA: Int = if (!transA) A.numCols else A.numRows + val kB: Int = if (!transB) B.numRows else B.numCols + val tAstr = if (!transA) "N" else "T" + val tBstr = if (!transB) "N" else "T" + + require(kA == kB, s"The columns of A don't match the rows of B. A: $kA, B: $kB") + require(mA == C.numRows, s"The rows of C don't match the rows of A. C: ${C.numRows}, A: $mA") + require(nB == C.numCols, + s"The columns of C don't match the columns of B. C: ${C.numCols}, A: $nB") + + nativeBLAS.dgemm(tAstr, tBstr, mA, nB, kA, alpha, A.values, A.numRows, B.values, B.numRows, + beta, C.values, C.numRows) + } + + /** + * C := alpha * A * B + beta * C + * For `SparseMatrix` A. + */ + private def gemm( + transA: Boolean, + transB: Boolean, + alpha: Double, + A: SparseMatrix, + B: DenseMatrix, + beta: Double, + C: DenseMatrix): Unit = { + val mA: Int = if (!transA) A.numRows else A.numCols + val nB: Int = if (!transB) B.numCols else B.numRows + val kA: Int = if (!transA) A.numCols else A.numRows + val kB: Int = if (!transB) B.numRows else B.numCols + + require(kA == kB, s"The columns of A don't match the rows of B. A: $kA, B: $kB") + require(mA == C.numRows, s"The rows of C don't match the rows of A. C: ${C.numRows}, A: $mA") + require(nB == C.numCols, + s"The columns of C don't match the columns of B. C: ${C.numCols}, A: $nB") + + val Avals = A.values + val Arows = if (!transA) A.rowIndices else A.colPtrs + val Acols = if (!transA) A.colPtrs else A.rowIndices + + // Slicing is easy in this case. This is the optimal multiplication setting for sparse matrices + if (transA){ + var colCounterForB = 0 + if (!transB) { // Expensive to put the check inside the loop + while (colCounterForB < nB) { + var rowCounterForA = 0 + val Cstart = colCounterForB * mA + val Bstart = colCounterForB * kA + while (rowCounterForA < mA) { + var i = Arows(rowCounterForA) + val indEnd = Arows(rowCounterForA + 1) + var sum = 0.0 + while (i < indEnd) { + sum += Avals(i) * B.values(Bstart + Acols(i)) + i += 1 + } + val Cindex = Cstart + rowCounterForA + C.values(Cindex) = beta * C.values(Cindex) + sum * alpha + rowCounterForA += 1 + } + colCounterForB += 1 + } + } else { + while (colCounterForB < nB) { + var rowCounter = 0 + val Cstart = colCounterForB * mA + while (rowCounter < mA) { + var i = Arows(rowCounter) + val indEnd = Arows(rowCounter + 1) + var sum = 0.0 + while (i < indEnd) { + sum += Avals(i) * B(colCounterForB, Acols(i)) + i += 1 + } + val Cindex = Cstart + rowCounter + C.values(Cindex) = beta * C.values(Cindex) + sum * alpha + rowCounter += 1 + } + colCounterForB += 1 + } + } + } else { + // Scale matrix first if `beta` is not equal to 0.0 + if (beta != 0.0){ + f2jBLAS.dscal(C.values.length, beta, C.values, 1) + } + // Perform matrix multiplication and add to C. The rows of A are multiplied by the columns of + // B, and added to C. + var colCounterForB = 0 // the column to be updated in C + if (!transB) { // Expensive to put the check inside the loop + while (colCounterForB < nB) { + var colCounterForA = 0 // The column of A to multiply with the row of B + val Bstart = colCounterForB * kB + val Cstart = colCounterForB * mA + while (colCounterForA < kA) { + var i = Acols(colCounterForA) + val indEnd = Acols(colCounterForA + 1) + val Bval = B.values(Bstart + colCounterForA) * alpha + while (i < indEnd){ + C.values(Cstart + Arows(i)) += Avals(i) * Bval + i += 1 + } + colCounterForA += 1 + } + colCounterForB += 1 + } + } else { + while (colCounterForB < nB) { + var colCounterForA = 0 // The column of A to multiply with the row of B + val Cstart = colCounterForB * mA + while (colCounterForA < kA){ + var i = Acols(colCounterForA) + val indEnd = Acols(colCounterForA + 1) + val Bval = B(colCounterForB, colCounterForA) * alpha + while (i < indEnd){ + C.values(Cstart + Arows(i)) += Avals(i) * Bval + i += 1 + } + colCounterForA += 1 + } + colCounterForB += 1 + } + } + } + } + + /** + * y := alpha * A * x + beta * y + * @param trans whether to use the transpose of matrix A (true), or A itself (false). + * @param alpha a scalar to scale the multiplication A * x. + * @param A the matrix A that will be left multiplied to x. Size of m x n. + * @param x the vector x that will be left multiplied by A. Size of n x 1. + * @param beta a scalar that can be used to scale vector y. + * @param y the resulting vector y. Size of m x 1. + */ + def gemv( + trans: Boolean, + alpha: Double, + A: Matrix, + x: DenseVector, + beta: Double, + y: DenseVector): Unit = { + + val mA: Int = if (!trans) A.numRows else A.numCols + val nx: Int = x.size + val nA: Int = if (!trans) A.numCols else A.numRows + + require(nA == nx, s"The columns of A don't match the number of elements of x. A: $nA, x: $nx") + require(mA == y.size, + s"The rows of A don't match the number of elements of y. A: $mA, y:${y.size}}") + if (alpha == 0.0) { + logDebug("gemv: alpha is equal to 0. Returning y.") + } else { + A match { + case sparse: SparseMatrix => + gemv(trans, alpha, sparse, x, beta, y) + case dense: DenseMatrix => + gemv(trans, alpha, dense, x, beta, y) + case _ => + throw new IllegalArgumentException(s"gemv doesn't support matrix type ${A.getClass}.") + } + } + } + + /** + * y := alpha * A * x + beta * y + * + * @param alpha a scalar to scale the multiplication A * x. + * @param A the matrix A that will be left multiplied to x. Size of m x n. + * @param x the vector x that will be left multiplied by A. Size of n x 1. + * @param beta a scalar that can be used to scale vector y. + * @param y the resulting vector y. Size of m x 1. + */ + def gemv( + alpha: Double, + A: Matrix, + x: DenseVector, + beta: Double, + y: DenseVector): Unit = { + gemv(false, alpha, A, x, beta, y) + } + + /** + * y := alpha * A * x + beta * y + * For `DenseMatrix` A. + */ + private def gemv( + trans: Boolean, + alpha: Double, + A: DenseMatrix, + x: DenseVector, + beta: Double, + y: DenseVector): Unit = { + val tStrA = if (!trans) "N" else "T" + nativeBLAS.dgemv(tStrA, A.numRows, A.numCols, alpha, A.values, A.numRows, x.values, 1, beta, + y.values, 1) + } + + /** + * y := alpha * A * x + beta * y + * For `SparseMatrix` A. + */ + private def gemv( + trans: Boolean, + alpha: Double, + A: SparseMatrix, + x: DenseVector, + beta: Double, + y: DenseVector): Unit = { + + val mA: Int = if(!trans) A.numRows else A.numCols + val nA: Int = if(!trans) A.numCols else A.numRows + + val Avals = A.values + val Arows = if (!trans) A.rowIndices else A.colPtrs + val Acols = if (!trans) A.colPtrs else A.rowIndices + + // Slicing is easy in this case. This is the optimal multiplication setting for sparse matrices + if (trans){ + var rowCounter = 0 + while (rowCounter < mA){ + var i = Arows(rowCounter) + val indEnd = Arows(rowCounter + 1) + var sum = 0.0 + while(i < indEnd){ + sum += Avals(i) * x.values(Acols(i)) + i += 1 + } + y.values(rowCounter) = beta * y.values(rowCounter) + sum * alpha + rowCounter += 1 + } + } else { + // Scale vector first if `beta` is not equal to 0.0 + if (beta != 0.0){ + scal(beta, y) + } + // Perform matrix-vector multiplication and add to y + var colCounterForA = 0 + while (colCounterForA < nA){ + var i = Acols(colCounterForA) + val indEnd = Acols(colCounterForA + 1) + val xVal = x.values(colCounterForA) * alpha + while (i < indEnd){ + val rowIndex = Arows(i) + y.values(rowIndex) += Avals(i) * xVal + i += 1 + } + colCounterForA += 1 + } + } + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index b11ba5d30fbd3..4e87fe088ecc5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -17,12 +17,16 @@ package org.apache.spark.mllib.linalg -import breeze.linalg.{Matrix => BM, DenseMatrix => BDM} +import java.util.Arrays + +import breeze.linalg.{Matrix => BM, DenseMatrix => BDM, CSCMatrix => BSM} + +import org.apache.spark.util.random.XORShiftRandom /** * Trait for a local matrix. */ -trait Matrix extends Serializable { +sealed trait Matrix extends Serializable { /** Number of rows. */ def numRows: Int @@ -37,8 +41,46 @@ trait Matrix extends Serializable { private[mllib] def toBreeze: BM[Double] /** Gets the (i, j)-th element. */ - private[mllib] def apply(i: Int, j: Int): Double = toBreeze(i, j) + private[mllib] def apply(i: Int, j: Int): Double + + /** Return the index for the (i, j)-th element in the backing array. */ + private[mllib] def index(i: Int, j: Int): Int + + /** Update element at (i, j) */ + private[mllib] def update(i: Int, j: Int, v: Double): Unit + + /** Get a deep copy of the matrix. */ + def copy: Matrix + /** Convenience method for `Matrix`-`DenseMatrix` multiplication. */ + def multiply(y: DenseMatrix): DenseMatrix = { + val C: DenseMatrix = Matrices.zeros(numRows, y.numCols).asInstanceOf[DenseMatrix] + BLAS.gemm(false, false, 1.0, this, y, 0.0, C) + C + } + + /** Convenience method for `Matrix`-`DenseVector` multiplication. */ + def multiply(y: DenseVector): DenseVector = { + val output = new DenseVector(new Array[Double](numRows)) + BLAS.gemv(1.0, this, y, 0.0, output) + output + } + + /** Convenience method for `Matrix`^T^-`DenseMatrix` multiplication. */ + def transposeMultiply(y: DenseMatrix): DenseMatrix = { + val C: DenseMatrix = Matrices.zeros(numCols, y.numCols).asInstanceOf[DenseMatrix] + BLAS.gemm(true, false, 1.0, this, y, 0.0, C) + C + } + + /** Convenience method for `Matrix`^T^-`DenseVector` multiplication. */ + def transposeMultiply(y: DenseVector): DenseVector = { + val output = new DenseVector(new Array[Double](numCols)) + BLAS.gemv(true, 1.0, this, y, 0.0, output) + output + } + + /** A human readable representation of the matrix */ override def toString: String = toBreeze.toString() } @@ -59,11 +101,104 @@ trait Matrix extends Serializable { */ class DenseMatrix(val numRows: Int, val numCols: Int, val values: Array[Double]) extends Matrix { - require(values.length == numRows * numCols) + require(values.length == numRows * numCols, "The number of values supplied doesn't match the " + + s"size of the matrix! values.length: ${values.length}, numRows * numCols: ${numRows * numCols}") override def toArray: Array[Double] = values - private[mllib] override def toBreeze: BM[Double] = new BDM[Double](numRows, numCols, values) + override def equals(o: Any) = o match { + case m: DenseMatrix => + m.numRows == numRows && m.numCols == numCols && Arrays.equals(toArray, m.toArray) + case _ => false + } + + private[mllib] def toBreeze: BM[Double] = new BDM[Double](numRows, numCols, values) + + private[mllib] def apply(i: Int): Double = values(i) + + private[mllib] def apply(i: Int, j: Int): Double = values(index(i, j)) + + private[mllib] def index(i: Int, j: Int): Int = i + numRows * j + + private[mllib] def update(i: Int, j: Int, v: Double): Unit = { + values(index(i, j)) = v + } + + override def copy = new DenseMatrix(numRows, numCols, values.clone()) +} + +/** + * Column-majored sparse matrix. + * The entry values are stored in Compressed Sparse Column (CSC) format. + * For example, the following matrix + * {{{ + * 1.0 0.0 4.0 + * 0.0 3.0 5.0 + * 2.0 0.0 6.0 + * }}} + * is stored as `values: [1.0, 2.0, 3.0, 4.0, 5.0, 6.0]`, + * `rowIndices=[0, 2, 1, 0, 1, 2]`, `colPointers=[0, 2, 3, 6]`. + * + * @param numRows number of rows + * @param numCols number of columns + * @param colPtrs the index corresponding to the start of a new column + * @param rowIndices the row index of the entry. They must be in strictly increasing order for each + * column + * @param values non-zero matrix entries in column major + */ +class SparseMatrix( + val numRows: Int, + val numCols: Int, + val colPtrs: Array[Int], + val rowIndices: Array[Int], + val values: Array[Double]) extends Matrix { + + require(values.length == rowIndices.length, "The number of row indices and values don't match! " + + s"values.length: ${values.length}, rowIndices.length: ${rowIndices.length}") + require(colPtrs.length == numCols + 1, "The length of the column indices should be the " + + s"number of columns + 1. Currently, colPointers.length: ${colPtrs.length}, " + + s"numCols: $numCols") + + override def toArray: Array[Double] = { + val arr = new Array[Double](numRows * numCols) + var j = 0 + while (j < numCols) { + var i = colPtrs(j) + val indEnd = colPtrs(j + 1) + val offset = j * numRows + while (i < indEnd) { + val rowIndex = rowIndices(i) + arr(offset + rowIndex) = values(i) + i += 1 + } + j += 1 + } + arr + } + + private[mllib] def toBreeze: BM[Double] = + new BSM[Double](values, numRows, numCols, colPtrs, rowIndices) + + private[mllib] def apply(i: Int, j: Int): Double = { + val ind = index(i, j) + if (ind < 0) 0.0 else values(ind) + } + + private[mllib] def index(i: Int, j: Int): Int = { + Arrays.binarySearch(rowIndices, colPtrs(j), colPtrs(j + 1), i) + } + + private[mllib] def update(i: Int, j: Int, v: Double): Unit = { + val ind = index(i, j) + if (ind == -1){ + throw new NoSuchElementException("The given row and column indices correspond to a zero " + + "value. Only non-zero elements in Sparse Matrices can be updated.") + } else { + values(index(i, j)) = v + } + } + + override def copy = new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.clone()) } /** @@ -82,6 +217,24 @@ object Matrices { new DenseMatrix(numRows, numCols, values) } + /** + * Creates a column-majored sparse matrix in Compressed Sparse Column (CSC) format. + * + * @param numRows number of rows + * @param numCols number of columns + * @param colPtrs the index corresponding to the start of a new column + * @param rowIndices the row index of the entry + * @param values non-zero matrix entries in column major + */ + def sparse( + numRows: Int, + numCols: Int, + colPtrs: Array[Int], + rowIndices: Array[Int], + values: Array[Double]): Matrix = { + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values) + } + /** * Creates a Matrix instance from a breeze matrix. * @param breeze a breeze matrix @@ -93,9 +246,84 @@ object Matrices { require(dm.majorStride == dm.rows, "Do not support stride size different from the number of rows.") new DenseMatrix(dm.rows, dm.cols, dm.data) + case sm: BSM[Double] => + new SparseMatrix(sm.rows, sm.cols, sm.colPtrs, sm.rowIndices, sm.data) case _ => throw new UnsupportedOperationException( s"Do not support conversion from type ${breeze.getClass.getName}.") } } + + /** + * Generate a `DenseMatrix` consisting of zeros. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `DenseMatrix` with size `numRows` x `numCols` and values of zeros + */ + def zeros(numRows: Int, numCols: Int): Matrix = + new DenseMatrix(numRows, numCols, new Array[Double](numRows * numCols)) + + /** + * Generate a `DenseMatrix` consisting of ones. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `DenseMatrix` with size `numRows` x `numCols` and values of ones + */ + def ones(numRows: Int, numCols: Int): Matrix = + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(1.0)) + + /** + * Generate an Identity Matrix in `DenseMatrix` format. + * @param n number of rows and columns of the matrix + * @return `DenseMatrix` with size `n` x `n` and values of ones on the diagonal + */ + def eye(n: Int): Matrix = { + val identity = Matrices.zeros(n, n) + var i = 0 + while (i < n){ + identity.update(i, i, 1.0) + i += 1 + } + identity + } + + /** + * Generate a `DenseMatrix` consisting of i.i.d. uniform random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1) + */ + def rand(numRows: Int, numCols: Int): Matrix = { + val rand = new XORShiftRandom + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rand.nextDouble())) + } + + /** + * Generate a `DenseMatrix` consisting of i.i.d. gaussian random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1) + */ + def randn(numRows: Int, numCols: Int): Matrix = { + val rand = new XORShiftRandom + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rand.nextGaussian())) + } + + /** + * Generate a diagonal matrix in `DenseMatrix` format from the supplied values. + * @param vector a `Vector` tat will form the values on the diagonal of the matrix + * @return Square `DenseMatrix` with size `values.length` x `values.length` and `values` + * on the diagonal + */ + def diag(vector: Vector): Matrix = { + val n = vector.size + val matrix = Matrices.eye(n) + val values = vector.toArray + var i = 0 + while (i < n) { + matrix.update(i, i, values(i)) + i += 1 + } + matrix + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index a45781d12e41e..6af225b7f49f7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -33,7 +33,7 @@ import org.apache.spark.SparkException * * Note: Users should not implement this interface. */ -trait Vector extends Serializable { +sealed trait Vector extends Serializable { /** * Size of the vector. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 478c6485052b6..66b58ba770160 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -106,19 +106,4 @@ class MatrixFactorizationModel private[mllib] ( } scored.top(num)(Ordering.by(_._2)) } - - /** - * :: DeveloperApi :: - * Predict the rating of many users for many products. - * This is a Java stub for python predictAll() - * - * @param usersProductsJRDD A JavaRDD with serialized tuples (user, product) - * @return JavaRDD of serialized Rating objects. - */ - @DeveloperApi - def predict(usersProductsJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = { - val usersProducts = usersProductsJRDD.rdd.map(xBytes => SerDe.unpackTuple(xBytes)) - predict(usersProducts).map(rate => SerDe.serializeRating(rate)) - } - } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala index 092d67bbc5238..db8ed62fa46ce 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala @@ -19,12 +19,15 @@ package org.apache.spark.mllib.api.python import org.scalatest.FunSuite -import org.apache.spark.mllib.linalg.{Matrices, Vectors} +import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices, Vectors} import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.recommendation.Rating class PythonMLLibAPISuite extends FunSuite { - test("vector serialization") { + SerDe.initialize() + + test("pickle vector") { val vectors = Seq( Vectors.dense(Array.empty[Double]), Vectors.dense(0.0), @@ -33,14 +36,13 @@ class PythonMLLibAPISuite extends FunSuite { Vectors.sparse(1, Array.empty[Int], Array.empty[Double]), Vectors.sparse(2, Array(1), Array(-2.0))) vectors.foreach { v => - val bytes = SerDe.serializeDoubleVector(v) - val u = SerDe.deserializeDoubleVector(bytes) + val u = SerDe.loads(SerDe.dumps(v)) assert(u.getClass === v.getClass) assert(u === v) } } - test("labeled point serialization") { + test("pickle labeled point") { val points = Seq( LabeledPoint(0.0, Vectors.dense(Array.empty[Double])), LabeledPoint(1.0, Vectors.dense(0.0)), @@ -49,34 +51,44 @@ class PythonMLLibAPISuite extends FunSuite { LabeledPoint(1.0, Vectors.sparse(1, Array.empty[Int], Array.empty[Double])), LabeledPoint(-0.5, Vectors.sparse(2, Array(1), Array(-2.0)))) points.foreach { p => - val bytes = SerDe.serializeLabeledPoint(p) - val q = SerDe.deserializeLabeledPoint(bytes) + val q = SerDe.loads(SerDe.dumps(p)).asInstanceOf[LabeledPoint] assert(q.label === p.label) assert(q.features.getClass === p.features.getClass) assert(q.features === p.features) } } - test("double serialization") { + test("pickle double") { for (x <- List(123.0, -10.0, 0.0, Double.MaxValue, Double.MinValue, Double.NaN)) { - val bytes = SerDe.serializeDouble(x) - val deser = SerDe.deserializeDouble(bytes) + val deser = SerDe.loads(SerDe.dumps(x.asInstanceOf[AnyRef])).asInstanceOf[Double] // We use `equals` here for comparison because we cannot use `==` for NaN assert(x.equals(deser)) } } - test("matrix to 2D array") { + test("pickle matrix") { val values = Array[Double](0, 1.2, 3, 4.56, 7, 8) val matrix = Matrices.dense(2, 3, values) - val arr = SerDe.to2dArray(matrix) - val expected = Array(Array[Double](0, 3, 7), Array[Double](1.2, 4.56, 8)) - assert(arr === expected) + val nm = SerDe.loads(SerDe.dumps(matrix)).asInstanceOf[DenseMatrix] + assert(matrix === nm) // Test conversion for empty matrix val empty = Array[Double]() val emptyMatrix = Matrices.dense(0, 0, empty) - val empty2D = SerDe.to2dArray(emptyMatrix) - assert(empty2D === Array[Array[Double]]()) + val ne = SerDe.loads(SerDe.dumps(emptyMatrix)).asInstanceOf[DenseMatrix] + assert(emptyMatrix == ne) + } + + test("pickle rating") { + val rat = new Rating(1, 2, 3.0) + val rat2 = SerDe.loads(SerDe.dumps(rat)).asInstanceOf[Rating] + assert(rat == rat2) + + // Test name of class only occur once + val rats = (1 to 10).map(x => new Rating(x, x + 1, x + 3.0)).toArray + val bytes = SerDe.dumps(rats) + assert(bytes.toString.split("Rating").length == 1) + assert(bytes.length / 10 < 25) // 25 bytes per rating + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala index 1952e6734ecf7..5d70c914f14b0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala @@ -126,4 +126,115 @@ class BLASSuite extends FunSuite { } } } + + test("gemm") { + + val dA = + new DenseMatrix(4, 3, Array(0.0, 1.0, 0.0, 0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 0.0, 3.0)) + val sA = new SparseMatrix(4, 3, Array(0, 1, 3, 4), Array(1, 0, 2, 3), Array(1.0, 2.0, 1.0, 3.0)) + + val B = new DenseMatrix(3, 2, Array(1.0, 0.0, 0.0, 0.0, 2.0, 1.0)) + val expected = new DenseMatrix(4, 2, Array(0.0, 1.0, 0.0, 0.0, 4.0, 0.0, 2.0, 3.0)) + + assert(dA multiply B ~== expected absTol 1e-15) + assert(sA multiply B ~== expected absTol 1e-15) + + val C1 = new DenseMatrix(4, 2, Array(1.0, 0.0, 2.0, 1.0, 0.0, 0.0, 1.0, 0.0)) + val C2 = C1.copy + val C3 = C1.copy + val C4 = C1.copy + val C5 = C1.copy + val C6 = C1.copy + val C7 = C1.copy + val C8 = C1.copy + val expected2 = new DenseMatrix(4, 2, Array(2.0, 1.0, 4.0, 2.0, 4.0, 0.0, 4.0, 3.0)) + val expected3 = new DenseMatrix(4, 2, Array(2.0, 2.0, 4.0, 2.0, 8.0, 0.0, 6.0, 6.0)) + + gemm(1.0, dA, B, 2.0, C1) + gemm(1.0, sA, B, 2.0, C2) + gemm(2.0, dA, B, 2.0, C3) + gemm(2.0, sA, B, 2.0, C4) + assert(C1 ~== expected2 absTol 1e-15) + assert(C2 ~== expected2 absTol 1e-15) + assert(C3 ~== expected3 absTol 1e-15) + assert(C4 ~== expected3 absTol 1e-15) + + withClue("columns of A don't match the rows of B") { + intercept[Exception] { + gemm(true, false, 1.0, dA, B, 2.0, C1) + } + } + + val dAT = + new DenseMatrix(3, 4, Array(0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 1.0, 0.0, 0.0, 0.0, 3.0)) + val sAT = + new SparseMatrix(3, 4, Array(0, 1, 2, 3, 4), Array(1, 0, 1, 2), Array(2.0, 1.0, 1.0, 3.0)) + + assert(dAT transposeMultiply B ~== expected absTol 1e-15) + assert(sAT transposeMultiply B ~== expected absTol 1e-15) + + gemm(true, false, 1.0, dAT, B, 2.0, C5) + gemm(true, false, 1.0, sAT, B, 2.0, C6) + gemm(true, false, 2.0, dAT, B, 2.0, C7) + gemm(true, false, 2.0, sAT, B, 2.0, C8) + assert(C5 ~== expected2 absTol 1e-15) + assert(C6 ~== expected2 absTol 1e-15) + assert(C7 ~== expected3 absTol 1e-15) + assert(C8 ~== expected3 absTol 1e-15) + } + + test("gemv") { + + val dA = + new DenseMatrix(4, 3, Array(0.0, 1.0, 0.0, 0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 0.0, 3.0)) + val sA = new SparseMatrix(4, 3, Array(0, 1, 3, 4), Array(1, 0, 2, 3), Array(1.0, 2.0, 1.0, 3.0)) + + val x = new DenseVector(Array(1.0, 2.0, 3.0)) + val expected = new DenseVector(Array(4.0, 1.0, 2.0, 9.0)) + + assert(dA multiply x ~== expected absTol 1e-15) + assert(sA multiply x ~== expected absTol 1e-15) + + val y1 = new DenseVector(Array(1.0, 3.0, 1.0, 0.0)) + val y2 = y1.copy + val y3 = y1.copy + val y4 = y1.copy + val y5 = y1.copy + val y6 = y1.copy + val y7 = y1.copy + val y8 = y1.copy + val expected2 = new DenseVector(Array(6.0, 7.0, 4.0, 9.0)) + val expected3 = new DenseVector(Array(10.0, 8.0, 6.0, 18.0)) + + gemv(1.0, dA, x, 2.0, y1) + gemv(1.0, sA, x, 2.0, y2) + gemv(2.0, dA, x, 2.0, y3) + gemv(2.0, sA, x, 2.0, y4) + assert(y1 ~== expected2 absTol 1e-15) + assert(y2 ~== expected2 absTol 1e-15) + assert(y3 ~== expected3 absTol 1e-15) + assert(y4 ~== expected3 absTol 1e-15) + withClue("columns of A don't match the rows of B") { + intercept[Exception] { + gemv(true, 1.0, dA, x, 2.0, y1) + } + } + + val dAT = + new DenseMatrix(3, 4, Array(0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 1.0, 0.0, 0.0, 0.0, 3.0)) + val sAT = + new SparseMatrix(3, 4, Array(0, 1, 2, 3, 4), Array(1, 0, 1, 2), Array(2.0, 1.0, 1.0, 3.0)) + + assert(dAT transposeMultiply x ~== expected absTol 1e-15) + assert(sAT transposeMultiply x ~== expected absTol 1e-15) + + gemv(true, 1.0, dAT, x, 2.0, y5) + gemv(true, 1.0, sAT, x, 2.0, y6) + gemv(true, 2.0, dAT, x, 2.0, y7) + gemv(true, 2.0, sAT, x, 2.0, y8) + assert(y5 ~== expected2 absTol 1e-15) + assert(y6 ~== expected2 absTol 1e-15) + assert(y7 ~== expected3 absTol 1e-15) + assert(y8 ~== expected3 absTol 1e-15) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala index 82d49c76ed02b..73a6d3a27d868 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.linalg import org.scalatest.FunSuite -import breeze.linalg.{DenseMatrix => BDM} +import breeze.linalg.{DenseMatrix => BDM, CSCMatrix => BSM} class BreezeMatrixConversionSuite extends FunSuite { test("dense matrix to breeze") { @@ -37,4 +37,26 @@ class BreezeMatrixConversionSuite extends FunSuite { assert(mat.numCols === breeze.cols) assert(mat.values.eq(breeze.data), "should not copy data") } + + test("sparse matrix to breeze") { + val values = Array(1.0, 2.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 1, 2) + val mat = Matrices.sparse(3, 2, colPtrs, rowIndices, values) + val breeze = mat.toBreeze.asInstanceOf[BSM[Double]] + assert(breeze.rows === mat.numRows) + assert(breeze.cols === mat.numCols) + assert(breeze.data.eq(mat.asInstanceOf[SparseMatrix].values), "should not copy data") + } + + test("sparse breeze matrix to sparse matrix") { + val values = Array(1.0, 2.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 1, 2) + val breeze = new BSM[Double](values, 3, 2, colPtrs, rowIndices) + val mat = Matrices.fromBreeze(breeze).asInstanceOf[SparseMatrix] + assert(mat.numRows === breeze.rows) + assert(mat.numCols === breeze.cols) + assert(mat.values.eq(breeze.data), "should not copy data") + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index 9c66b4db9f16b..5f8b8c4b72697 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -36,4 +36,80 @@ class MatricesSuite extends FunSuite { Matrices.dense(3, 2, Array(0.0, 1.0, 2.0)) } } + + test("sparse matrix construction") { + val m = 3 + val n = 2 + val values = Array(1.0, 2.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 1, 2) + val mat = Matrices.sparse(m, n, colPtrs, rowIndices, values).asInstanceOf[SparseMatrix] + assert(mat.numRows === m) + assert(mat.numCols === n) + assert(mat.values.eq(values), "should not copy data") + assert(mat.colPtrs.eq(colPtrs), "should not copy data") + assert(mat.rowIndices.eq(rowIndices), "should not copy data") + } + + test("sparse matrix construction with wrong number of elements") { + intercept[IllegalArgumentException] { + Matrices.sparse(3, 2, Array(0, 1), Array(1, 2, 1), Array(0.0, 1.0, 2.0)) + } + + intercept[IllegalArgumentException] { + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(0.0, 1.0, 2.0)) + } + } + + test("matrix copies are deep copies") { + val m = 3 + val n = 2 + + val denseMat = Matrices.dense(m, n, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val denseCopy = denseMat.copy + + assert(!denseMat.toArray.eq(denseCopy.toArray)) + + val values = Array(1.0, 2.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 1, 2) + val sparseMat = Matrices.sparse(m, n, colPtrs, rowIndices, values) + val sparseCopy = sparseMat.copy + + assert(!sparseMat.toArray.eq(sparseCopy.toArray)) + } + + test("matrix indexing and updating") { + val m = 3 + val n = 2 + val allValues = Array(0.0, 1.0, 2.0, 3.0, 4.0, 0.0) + + val denseMat = new DenseMatrix(m, n, allValues) + + assert(denseMat(0, 1) === 3.0) + assert(denseMat(0, 1) === denseMat.values(3)) + assert(denseMat(0, 1) === denseMat(3)) + assert(denseMat(0, 0) === 0.0) + + denseMat.update(0, 0, 10.0) + assert(denseMat(0, 0) === 10.0) + assert(denseMat.values(0) === 10.0) + + val sparseValues = Array(1.0, 2.0, 3.0, 4.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 0, 1) + val sparseMat = new SparseMatrix(m, n, colPtrs, rowIndices, sparseValues) + + assert(sparseMat(0, 1) === 3.0) + assert(sparseMat(0, 1) === sparseMat.values(2)) + assert(sparseMat(0, 0) === 0.0) + + intercept[NoSuchElementException] { + sparseMat.update(0, 0, 10.0) + } + + sparseMat.update(0, 1, 10.0) + assert(sparseMat(0, 1) === 10.0) + assert(sparseMat.values(2) === 10.0) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala index 29cc42d8cbea7..30b906aaa3ba4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.util -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.{Matrix, Vector} import org.scalatest.exceptions.TestFailedException object TestingUtils { @@ -169,4 +169,67 @@ object TestingUtils { override def toString = x.toString } + case class CompareMatrixRightSide( + fun: (Matrix, Matrix, Double) => Boolean, y: Matrix, eps: Double, method: String) + + /** + * Implicit class for comparing two matrices using relative tolerance or absolute tolerance. + */ + implicit class MatrixWithAlmostEquals(val x: Matrix) { + + /** + * When the difference of two vectors are within eps, returns true; otherwise, returns false. + */ + def ~=(r: CompareMatrixRightSide): Boolean = r.fun(x, r.y, r.eps) + + /** + * When the difference of two vectors are within eps, returns false; otherwise, returns true. + */ + def !~=(r: CompareMatrixRightSide): Boolean = !r.fun(x, r.y, r.eps) + + /** + * Throws exception when the difference of two vectors are NOT within eps; + * otherwise, returns true. + */ + def ~==(r: CompareMatrixRightSide): Boolean = { + if (!r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Expected \n$x\n and \n${r.y}\n to be within ${r.eps}${r.method} for all elements.", 0) + } + true + } + + /** + * Throws exception when the difference of two matrices are within eps; otherwise, returns true. + */ + def !~==(r: CompareMatrixRightSide): Boolean = { + if (r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Did not expect \n$x\n and \n${r.y}\n to be within " + + "${r.eps}${r.method} for all elements.", 0) + } + true + } + + /** + * Comparison using absolute tolerance. + */ + def absTol(eps: Double): CompareMatrixRightSide = CompareMatrixRightSide( + (x: Matrix, y: Matrix, eps: Double) => { + x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 absTol eps) + }, x, eps, ABS_TOL_MSG) + + /** + * Comparison using relative tolerance. Note that comparing against sparse vector + * with elements having value of zero will raise exception because it involves with + * comparing against zero. + */ + def relTol(eps: Double): CompareMatrixRightSide = CompareMatrixRightSide( + (x: Matrix, y: Matrix, eps: Double) => { + x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) + }, x, eps, REL_TOL_MSG) + + override def toString = x.toString + } + } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 2f1e05dfcc7b1..3280e662fa0b1 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -37,7 +37,9 @@ object MimaExcludes { Seq( MimaBuild.excludeSparkPackage("deploy"), MimaBuild.excludeSparkPackage("graphx") - ) + ) ++ + MimaBuild.excludeSparkClass("mllib.linalg.Matrix") ++ + MimaBuild.excludeSparkClass("mllib.linalg.Vector") case v if v.startsWith("1.1") => Seq( diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ab9f8ba120e83..12ac82293df76 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -336,7 +336,7 @@ object TestSettings { fork := true, javaOptions in Test += "-Dspark.test.home=" + sparkHome, javaOptions in Test += "-Dspark.testing=1", - javaOptions in Test += "-Dspark.ports.maxRetries=100", + javaOptions in Test += "-Dspark.port.maxRetries=100", javaOptions in Test += "-Dspark.ui.enabled=false", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark") diff --git a/python/epydoc.conf b/python/epydoc.conf index 51c0faf359939..8593e08deda19 100644 --- a/python/epydoc.conf +++ b/python/epydoc.conf @@ -34,5 +34,5 @@ private: no exclude: pyspark.cloudpickle pyspark.worker pyspark.join pyspark.java_gateway pyspark.examples pyspark.shell pyspark.tests - pyspark.rddsampler pyspark.daemon pyspark.mllib._common + pyspark.rddsampler pyspark.daemon pyspark.mllib.tests pyspark.shuffle diff --git a/python/pyspark/context.py b/python/pyspark/context.py index a17f2c1203d36..064a24bff539c 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -211,6 +211,7 @@ def _ensure_initialized(cls, instance=None, gateway=None): SparkContext._jvm = SparkContext._gateway.jvm SparkContext._writeToFile = SparkContext._jvm.PythonRDD.writeToFile SparkContext._jvm.SerDeUtil.initialize() + SparkContext._jvm.SerDe.initialize() if instance: if (SparkContext._active_spark_context and diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py deleted file mode 100644 index 68f6033616726..0000000000000 --- a/python/pyspark/mllib/_common.py +++ /dev/null @@ -1,562 +0,0 @@ -# -# 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. -# - -import struct -import sys -import numpy -from numpy import ndarray, float64, int64, int32, array_equal, array -from pyspark import SparkContext, RDD -from pyspark.mllib.linalg import SparseVector -from pyspark.serializers import FramedSerializer - - -""" -Common utilities shared throughout MLlib, primarily for dealing with -different data types. These include: -- Serialization utilities to / from byte arrays that Java can handle -- Serializers for other data types, like ALS Rating objects -- Common methods for linear models -- Methods to deal with the different vector types we support, such as - SparseVector and scipy.sparse matrices. -""" - - -# Check whether we have SciPy. MLlib works without it too, but if we have it, some methods, -# such as _dot and _serialize_double_vector, start to support scipy.sparse matrices. - -_have_scipy = False -_scipy_issparse = None -try: - import scipy.sparse - _have_scipy = True - _scipy_issparse = scipy.sparse.issparse -except: - # No SciPy in environment, but that's okay - pass - - -# Serialization functions to and from Scala. These use the following formats, understood -# by the PythonMLLibAPI class in Scala: -# -# Dense double vector format: -# -# [1-byte 1] [4-byte length] [length*8 bytes of data] -# -# Sparse double vector format: -# -# [1-byte 2] [4-byte length] [4-byte nonzeros] [nonzeros*4 bytes of indices] \ -# [nonzeros*8 bytes of values] -# -# Double matrix format: -# -# [1-byte 3] [4-byte rows] [4-byte cols] [rows*cols*8 bytes of data] -# -# LabeledPoint format: -# -# [1-byte 4] [8-byte label] [dense or sparse vector] -# -# This is all in machine-endian. That means that the Java interpreter and the -# Python interpreter must agree on what endian the machine is. - - -DENSE_VECTOR_MAGIC = 1 -SPARSE_VECTOR_MAGIC = 2 -DENSE_MATRIX_MAGIC = 3 -LABELED_POINT_MAGIC = 4 - - -# Workaround for SPARK-2954: before Python 2.7, struct.unpack couldn't unpack bytearray()s. -if sys.version_info[:2] <= (2, 6): - def _unpack(fmt, string): - return struct.unpack(fmt, buffer(string)) -else: - _unpack = struct.unpack - - -def _deserialize_numpy_array(shape, ba, offset, dtype=float64): - """ - Deserialize a numpy array of the given type from an offset in - bytearray ba, assigning it the given shape. - - >>> x = array([1.0, 2.0, 3.0, 4.0, 5.0]) - >>> array_equal(x, _deserialize_numpy_array(x.shape, x.data, 0)) - True - >>> x = array([1.0, 2.0, 3.0, 4.0]).reshape(2,2) - >>> array_equal(x, _deserialize_numpy_array(x.shape, x.data, 0)) - True - >>> x = array([1, 2, 3], dtype=int32) - >>> array_equal(x, _deserialize_numpy_array(x.shape, x.data, 0, dtype=int32)) - True - """ - ar = ndarray(shape=shape, buffer=ba, offset=offset, dtype=dtype, order='C') - return ar.copy() - - -def _serialize_double(d): - """ - Serialize a double (float or numpy.float64) into a mutually understood format. - """ - if type(d) == float or type(d) == float64 or type(d) == int or type(d) == long: - d = float64(d) - ba = bytearray(8) - _copyto(d, buffer=ba, offset=0, shape=[1], dtype=float64) - return ba - else: - raise TypeError("_serialize_double called on non-float input") - - -def _serialize_double_vector(v): - """ - Serialize a double vector into a mutually understood format. - - Note: we currently do not use a magic byte for double for storage - efficiency. This should be reconsidered when we add Ser/De for other - 8-byte types (e.g. Long), for safety. The corresponding deserializer, - _deserialize_double, needs to be modified as well if the serialization - scheme changes. - - >>> x = array([1,2,3]) - >>> y = _deserialize_double_vector(_serialize_double_vector(x)) - >>> array_equal(y, array([1.0, 2.0, 3.0])) - True - """ - v = _convert_vector(v) - if type(v) == ndarray: - return _serialize_dense_vector(v) - elif type(v) == SparseVector: - return _serialize_sparse_vector(v) - else: - raise TypeError("_serialize_double_vector called on a %s; " - "wanted ndarray or SparseVector" % type(v)) - - -def _serialize_dense_vector(v): - """Serialize a dense vector given as a NumPy array.""" - if v.ndim != 1: - raise TypeError("_serialize_double_vector called on a %ddarray; " - "wanted a 1darray" % v.ndim) - if v.dtype != float64: - if numpy.issubdtype(v.dtype, numpy.complex): - raise TypeError("_serialize_double_vector called on an ndarray of %s; " - "wanted ndarray of float64" % v.dtype) - v = v.astype(float64) - length = v.shape[0] - ba = bytearray(5 + 8 * length) - ba[0] = DENSE_VECTOR_MAGIC - length_bytes = ndarray(shape=[1], buffer=ba, offset=1, dtype=int32) - length_bytes[0] = length - _copyto(v, buffer=ba, offset=5, shape=[length], dtype=float64) - return ba - - -def _serialize_sparse_vector(v): - """Serialize a pyspark.mllib.linalg.SparseVector.""" - nonzeros = len(v.indices) - ba = bytearray(9 + 12 * nonzeros) - ba[0] = SPARSE_VECTOR_MAGIC - header = ndarray(shape=[2], buffer=ba, offset=1, dtype=int32) - header[0] = v.size - header[1] = nonzeros - _copyto(v.indices, buffer=ba, offset=9, shape=[nonzeros], dtype=int32) - values_offset = 9 + 4 * nonzeros - _copyto(v.values, buffer=ba, offset=values_offset, shape=[nonzeros], dtype=float64) - return ba - - -def _deserialize_double(ba, offset=0): - """Deserialize a double from a mutually understood format. - - >>> import sys - >>> _deserialize_double(_serialize_double(123.0)) == 123.0 - True - >>> _deserialize_double(_serialize_double(float64(0.0))) == 0.0 - True - >>> _deserialize_double(_serialize_double(1)) == 1.0 - True - >>> _deserialize_double(_serialize_double(1L)) == 1.0 - True - >>> x = sys.float_info.max - >>> _deserialize_double(_serialize_double(sys.float_info.max)) == x - True - >>> y = float64(sys.float_info.max) - >>> _deserialize_double(_serialize_double(sys.float_info.max)) == y - True - """ - if type(ba) != bytearray: - raise TypeError("_deserialize_double called on a %s; wanted bytearray" % type(ba)) - if len(ba) - offset != 8: - raise TypeError("_deserialize_double called on a %d-byte array; wanted 8 bytes." % nb) - return _unpack("d", ba[offset:])[0] - - -def _deserialize_double_vector(ba, offset=0): - """Deserialize a double vector from a mutually understood format. - - >>> x = array([1.0, 2.0, 3.0, 4.0, -1.0, 0.0, -0.0]) - >>> array_equal(x, _deserialize_double_vector(_serialize_double_vector(x))) - True - >>> s = SparseVector(4, [1, 3], [3.0, 5.5]) - >>> s == _deserialize_double_vector(_serialize_double_vector(s)) - True - """ - if type(ba) != bytearray: - raise TypeError("_deserialize_double_vector called on a %s; " - "wanted bytearray" % type(ba)) - nb = len(ba) - offset - if nb < 5: - raise TypeError("_deserialize_double_vector called on a %d-byte array, " - "which is too short" % nb) - if ba[offset] == DENSE_VECTOR_MAGIC: - return _deserialize_dense_vector(ba, offset) - elif ba[offset] == SPARSE_VECTOR_MAGIC: - return _deserialize_sparse_vector(ba, offset) - else: - raise TypeError("_deserialize_double_vector called on bytearray " - "with wrong magic") - - -def _deserialize_dense_vector(ba, offset=0): - """Deserialize a dense vector into a numpy array.""" - nb = len(ba) - offset - if nb < 5: - raise TypeError("_deserialize_dense_vector called on a %d-byte array, " - "which is too short" % nb) - length = ndarray(shape=[1], buffer=ba, offset=offset + 1, dtype=int32)[0] - if nb < 8 * length + 5: - raise TypeError("_deserialize_dense_vector called on bytearray " - "with wrong length") - return _deserialize_numpy_array([length], ba, offset + 5) - - -def _deserialize_sparse_vector(ba, offset=0): - """Deserialize a sparse vector into a MLlib SparseVector object.""" - nb = len(ba) - offset - if nb < 9: - raise TypeError("_deserialize_sparse_vector called on a %d-byte array, " - "which is too short" % nb) - header = ndarray(shape=[2], buffer=ba, offset=offset + 1, dtype=int32) - size = header[0] - nonzeros = header[1] - if nb < 9 + 12 * nonzeros: - raise TypeError("_deserialize_sparse_vector called on bytearray " - "with wrong length") - indices = _deserialize_numpy_array([nonzeros], ba, offset + 9, dtype=int32) - values = _deserialize_numpy_array([nonzeros], ba, offset + 9 + 4 * nonzeros, dtype=float64) - return SparseVector(int(size), indices, values) - - -def _serialize_double_matrix(m): - """Serialize a double matrix into a mutually understood format.""" - if (type(m) == ndarray and m.ndim == 2): - if m.dtype != float64: - if numpy.issubdtype(m.dtype, numpy.complex): - raise TypeError("_serialize_double_matrix called on an ndarray of %s; " - "wanted ndarray of float64" % m.dtype) - m = m.astype(float64) - rows = m.shape[0] - cols = m.shape[1] - ba = bytearray(9 + 8 * rows * cols) - ba[0] = DENSE_MATRIX_MAGIC - lengths = ndarray(shape=[3], buffer=ba, offset=1, dtype=int32) - lengths[0] = rows - lengths[1] = cols - _copyto(m, buffer=ba, offset=9, shape=[rows, cols], dtype=float64) - return ba - else: - raise TypeError("_serialize_double_matrix called on a " - "non-double-matrix") - - -def _deserialize_double_matrix(ba): - """Deserialize a double matrix from a mutually understood format.""" - if type(ba) != bytearray: - raise TypeError("_deserialize_double_matrix called on a %s; " - "wanted bytearray" % type(ba)) - if len(ba) < 9: - raise TypeError("_deserialize_double_matrix called on a %d-byte array, " - "which is too short" % len(ba)) - if ba[0] != DENSE_MATRIX_MAGIC: - raise TypeError("_deserialize_double_matrix called on bytearray " - "with wrong magic") - lengths = ndarray(shape=[2], buffer=ba, offset=1, dtype=int32) - rows = lengths[0] - cols = lengths[1] - if (len(ba) != 8 * rows * cols + 9): - raise TypeError("_deserialize_double_matrix called on bytearray " - "with wrong length") - return _deserialize_numpy_array([rows, cols], ba, 9) - - -def _serialize_labeled_point(p): - """ - Serialize a LabeledPoint with a features vector of any type. - - >>> from pyspark.mllib.regression import LabeledPoint - >>> dp0 = LabeledPoint(0.5, array([1.0, 2.0, 3.0, 4.0, -1.0, 0.0, -0.0])) - >>> dp1 = _deserialize_labeled_point(_serialize_labeled_point(dp0)) - >>> dp1.label == dp0.label - True - >>> array_equal(dp1.features, dp0.features) - True - >>> sp0 = LabeledPoint(0.0, SparseVector(4, [1, 3], [3.0, 5.5])) - >>> sp1 = _deserialize_labeled_point(_serialize_labeled_point(sp0)) - >>> sp1.label == sp1.label - True - >>> sp1.features == sp0.features - True - """ - from pyspark.mllib.regression import LabeledPoint - serialized_features = _serialize_double_vector(p.features) - header = bytearray(9) - header[0] = LABELED_POINT_MAGIC - header_float = ndarray(shape=[1], buffer=header, offset=1, dtype=float64) - header_float[0] = p.label - return header + serialized_features - - -def _deserialize_labeled_point(ba, offset=0): - """Deserialize a LabeledPoint from a mutually understood format.""" - from pyspark.mllib.regression import LabeledPoint - if type(ba) != bytearray: - raise TypeError("Expecting a bytearray but got %s" % type(ba)) - if ba[offset] != LABELED_POINT_MAGIC: - raise TypeError("Expecting magic number %d but got %d" % (LABELED_POINT_MAGIC, ba[0])) - label = ndarray(shape=[1], buffer=ba, offset=offset + 1, dtype=float64)[0] - features = _deserialize_double_vector(ba, offset + 9) - return LabeledPoint(label, features) - - -def _copyto(array, buffer, offset, shape, dtype): - """ - Copy the contents of a vector to a destination bytearray at the - given offset. - - TODO: In the future this could use numpy.copyto on NumPy 1.7+, but - we should benchmark that to see whether it provides a benefit. - """ - temp_array = ndarray(shape=shape, buffer=buffer, offset=offset, dtype=dtype, order='C') - temp_array[...] = array - - -def _get_unmangled_rdd(data, serializer, cache=True): - """ - :param cache: If True, the serialized RDD is cached. (default = True) - WARNING: Users should unpersist() this later! - """ - dataBytes = data.map(serializer) - dataBytes._bypass_serializer = True - if cache: - dataBytes.cache() - return dataBytes - - -def _get_unmangled_double_vector_rdd(data, cache=True): - """ - Map a pickled Python RDD of Python dense or sparse vectors to a Java RDD of - _serialized_double_vectors. - :param cache: If True, the serialized RDD is cached. (default = True) - WARNING: Users should unpersist() this later! - """ - return _get_unmangled_rdd(data, _serialize_double_vector, cache) - - -def _get_unmangled_labeled_point_rdd(data, cache=True): - """ - Map a pickled Python RDD of LabeledPoint to a Java RDD of _serialized_labeled_points. - :param cache: If True, the serialized RDD is cached. (default = True) - WARNING: Users should unpersist() this later! - """ - return _get_unmangled_rdd(data, _serialize_labeled_point, cache) - - -# Common functions for dealing with and training linear models - -def _linear_predictor_typecheck(x, coeffs): - """ - Check that x is a one-dimensional vector of the right shape. - This is a temporary hackaround until we actually implement bulk predict. - """ - x = _convert_vector(x) - if type(x) == ndarray: - if x.ndim == 1: - if x.shape != coeffs.shape: - raise RuntimeError("Got array of %d elements; wanted %d" % ( - numpy.shape(x)[0], coeffs.shape[0])) - else: - raise RuntimeError("Bulk predict not yet supported.") - elif type(x) == SparseVector: - if x.size != coeffs.shape[0]: - raise RuntimeError("Got sparse vector of size %d; wanted %d" % ( - x.size, coeffs.shape[0])) - elif isinstance(x, RDD): - raise RuntimeError("Bulk predict not yet supported.") - else: - raise TypeError("Argument of type " + type(x).__name__ + " unsupported") - - -# If we weren't given initial weights, take a zero vector of the appropriate -# length. -def _get_initial_weights(initial_weights, data): - if initial_weights is None: - initial_weights = _convert_vector(data.first().features) - if type(initial_weights) == ndarray: - if initial_weights.ndim != 1: - raise TypeError("At least one data element has " - + initial_weights.ndim + " dimensions, which is not 1") - initial_weights = numpy.zeros([initial_weights.shape[0]]) - elif type(initial_weights) == SparseVector: - initial_weights = numpy.zeros([initial_weights.size]) - return initial_weights - - -# train_func should take two parameters, namely data and initial_weights, and -# return the result of a call to the appropriate JVM stub. -# _regression_train_wrapper is responsible for setup and error checking. -def _regression_train_wrapper(sc, train_func, klass, data, initial_weights): - initial_weights = _get_initial_weights(initial_weights, data) - dataBytes = _get_unmangled_labeled_point_rdd(data) - ans = train_func(dataBytes, _serialize_double_vector(initial_weights)) - if len(ans) != 2: - raise RuntimeError("JVM call result had unexpected length") - elif type(ans[0]) != bytearray: - raise RuntimeError("JVM call result had first element of type " - + type(ans[0]).__name__ + " which is not bytearray") - elif type(ans[1]) != float: - raise RuntimeError("JVM call result had second element of type " - + type(ans[0]).__name__ + " which is not float") - return klass(_deserialize_double_vector(ans[0]), ans[1]) - - -# Functions for serializing ALS Rating objects and tuples - -def _serialize_rating(r): - ba = bytearray(16) - intpart = ndarray(shape=[2], buffer=ba, dtype=int32) - doublepart = ndarray(shape=[1], buffer=ba, dtype=float64, offset=8) - intpart[0], intpart[1], doublepart[0] = r - return ba - - -class RatingDeserializer(FramedSerializer): - - def loads(self, string): - res = ndarray(shape=(3, ), buffer=string, dtype=float64, offset=4) - return int(res[0]), int(res[1]), res[2] - - def load_stream(self, stream): - while True: - try: - yield self._read_with_length(stream) - except struct.error: - return - except EOFError: - return - - -def _serialize_tuple(t): - ba = bytearray(8) - intpart = ndarray(shape=[2], buffer=ba, dtype=int32) - intpart[0], intpart[1] = t - return ba - - -# Vector math functions that support all of our vector types - -def _convert_vector(vec): - """ - Convert a vector to a format we support internally. This does - the following: - - * For dense NumPy vectors (ndarray), returns them as is - * For our SparseVector class, returns that as is - * For Python lists, converts them to NumPy vectors - * For scipy.sparse.*_matrix column vectors, converts them to - our own SparseVector type. - - This should be called before passing any data to our algorithms - or attempting to serialize it to Java. - """ - if type(vec) == ndarray or type(vec) == SparseVector: - return vec - elif type(vec) == list: - return array(vec, dtype=float64) - elif _have_scipy: - if _scipy_issparse(vec): - assert vec.shape[1] == 1, "Expected column vector" - csc = vec.tocsc() - return SparseVector(vec.shape[0], csc.indices, csc.data) - raise TypeError("Expected NumPy array, SparseVector, or scipy.sparse matrix") - - -def _squared_distance(v1, v2): - """ - Squared distance of two NumPy or sparse vectors. - - >>> dense1 = array([1., 2.]) - >>> sparse1 = SparseVector(2, [0, 1], [1., 2.]) - >>> dense2 = array([2., 1.]) - >>> sparse2 = SparseVector(2, [0, 1], [2., 1.]) - >>> _squared_distance(dense1, dense2) - 2.0 - >>> _squared_distance(dense1, sparse2) - 2.0 - >>> _squared_distance(sparse1, dense2) - 2.0 - >>> _squared_distance(sparse1, sparse2) - 2.0 - """ - v1 = _convert_vector(v1) - v2 = _convert_vector(v2) - if type(v1) == ndarray and type(v2) == ndarray: - diff = v1 - v2 - return numpy.dot(diff, diff) - elif type(v1) == ndarray: - return v2.squared_distance(v1) - else: - return v1.squared_distance(v2) - - -def _dot(vec, target): - """ - Compute the dot product of a vector of the types we support - (Numpy array, list, SparseVector, or SciPy sparse) and a target - NumPy array that is either 1- or 2-dimensional. Equivalent to - calling numpy.dot of the two vectors, but for SciPy ones, we - have to transpose them because they're column vectors. - """ - if type(vec) == ndarray: - return numpy.dot(vec, target) - elif type(vec) == SparseVector: - return vec.dot(target) - elif type(vec) == list: - return numpy.dot(_convert_vector(vec), target) - else: - return vec.transpose().dot(target)[0] - - -def _test(): - import doctest - globs = globals().copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) - (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() - if failure_count: - exit(-1) - - -if __name__ == "__main__": - _test() diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 71ab46b61d7fa..ac142fb49a90c 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -15,19 +15,14 @@ # limitations under the License. # +from math import exp + import numpy +from numpy import array -from numpy import array, shape -from pyspark import SparkContext -from pyspark.mllib._common import \ - _dot, _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \ - _serialize_double_matrix, _deserialize_double_matrix, \ - _serialize_double_vector, _deserialize_double_vector, \ - _get_initial_weights, _serialize_rating, _regression_train_wrapper, \ - _linear_predictor_typecheck, _get_unmangled_labeled_point_rdd -from pyspark.mllib.linalg import SparseVector -from pyspark.mllib.regression import LabeledPoint, LinearModel -from math import exp, log +from pyspark import SparkContext, PickleSerializer +from pyspark.mllib.linalg import SparseVector, _convert_to_vector +from pyspark.mllib.regression import LabeledPoint, LinearModel, _regression_train_wrapper __all__ = ['LogisticRegressionModel', 'LogisticRegressionWithSGD', 'SVMModel', @@ -67,8 +62,7 @@ class LogisticRegressionModel(LinearModel): """ def predict(self, x): - _linear_predictor_typecheck(x, self._coeff) - margin = _dot(x, self._coeff) + self._intercept + margin = self.weights.dot(x) + self._intercept if margin > 0: prob = 1 / (1 + exp(-margin)) else: @@ -81,7 +75,7 @@ class LogisticRegressionWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, - initialWeights=None, regParam=1.0, regType=None, intercept=False): + initialWeights=None, regParam=1.0, regType="none", intercept=False): """ Train a logistic regression model on the given data. @@ -106,11 +100,12 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, are activated or not). """ sc = data.context - if regType is None: - regType = "none" - train_func = lambda d, i: sc._jvm.PythonMLLibAPI().trainLogisticRegressionModelWithSGD( - d._jrdd, iterations, step, miniBatchFraction, i, regParam, regType, intercept) - return _regression_train_wrapper(sc, train_func, LogisticRegressionModel, data, + + def train(jdata, i): + return sc._jvm.PythonMLLibAPI().trainLogisticRegressionModelWithSGD( + jdata, iterations, step, miniBatchFraction, i, regParam, regType, intercept) + + return _regression_train_wrapper(sc, train, LogisticRegressionModel, data, initialWeights) @@ -141,8 +136,7 @@ class SVMModel(LinearModel): """ def predict(self, x): - _linear_predictor_typecheck(x, self._coeff) - margin = _dot(x, self._coeff) + self._intercept + margin = self.weights.dot(x) + self.intercept return 1 if margin >= 0 else 0 @@ -150,7 +144,7 @@ class SVMWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, regParam=1.0, - miniBatchFraction=1.0, initialWeights=None, regType=None, intercept=False): + miniBatchFraction=1.0, initialWeights=None, regType="none", intercept=False): """ Train a support vector machine on the given data. @@ -175,11 +169,12 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, are activated or not). """ sc = data.context - if regType is None: - regType = "none" - train_func = lambda d, i: sc._jvm.PythonMLLibAPI().trainSVMModelWithSGD( - d._jrdd, iterations, step, regParam, miniBatchFraction, i, regType, intercept) - return _regression_train_wrapper(sc, train_func, SVMModel, data, initialWeights) + + def train(jrdd, i): + return sc._jvm.PythonMLLibAPI().trainSVMModelWithSGD( + jrdd, iterations, step, regParam, miniBatchFraction, i, regType, intercept) + + return _regression_train_wrapper(sc, train, SVMModel, data, initialWeights) class NaiveBayesModel(object): @@ -220,7 +215,8 @@ def __init__(self, labels, pi, theta): def predict(self, x): """Return the most likely class for a data vector x""" - return self.labels[numpy.argmax(self.pi + _dot(x, self.theta.transpose()))] + x = _convert_to_vector(x) + return self.labels[numpy.argmax(self.pi + x.dot(self.theta.transpose()))] class NaiveBayes(object): @@ -242,12 +238,9 @@ def train(cls, data, lambda_=1.0): @param lambda_: The smoothing parameter """ sc = data.context - dataBytes = _get_unmangled_labeled_point_rdd(data) - ans = sc._jvm.PythonMLLibAPI().trainNaiveBayes(dataBytes._jrdd, lambda_) - return NaiveBayesModel( - _deserialize_double_vector(ans[0]), - _deserialize_double_vector(ans[1]), - _deserialize_double_matrix(ans[2])) + jlist = sc._jvm.PythonMLLibAPI().trainNaiveBayes(data._to_java_object_rdd(), lambda_) + labels, pi, theta = PickleSerializer().loads(str(sc._jvm.SerDe.dumps(jlist))) + return NaiveBayesModel(labels.toArray(), pi.toArray(), numpy.array(theta)) def _test(): diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index f3e952a1d842a..12c56022717a5 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -15,15 +15,9 @@ # limitations under the License. # -from numpy import array, dot -from math import sqrt from pyspark import SparkContext -from pyspark.mllib._common import \ - _get_unmangled_rdd, _get_unmangled_double_vector_rdd, _squared_distance, \ - _serialize_double_matrix, _deserialize_double_matrix, \ - _serialize_double_vector, _deserialize_double_vector, \ - _get_initial_weights, _serialize_rating, _regression_train_wrapper -from pyspark.mllib.linalg import SparseVector +from pyspark.serializers import PickleSerializer, AutoBatchedSerializer +from pyspark.mllib.linalg import SparseVector, _convert_to_vector __all__ = ['KMeansModel', 'KMeans'] @@ -32,6 +26,7 @@ class KMeansModel(object): """A clustering model derived from the k-means method. + >>> from numpy import array >>> data = array([0.0,0.0, 1.0,1.0, 9.0,8.0, 8.0,9.0]).reshape(4,2) >>> model = KMeans.train( ... sc.parallelize(data), 2, maxIterations=10, runs=30, initializationMode="random") @@ -71,8 +66,9 @@ def predict(self, x): """Find the cluster to which x belongs in this model.""" best = 0 best_distance = float("inf") - for i in range(0, len(self.centers)): - distance = _squared_distance(x, self.centers[i]) + x = _convert_to_vector(x) + for i in xrange(len(self.centers)): + distance = x.squared_distance(self.centers[i]) if distance < best_distance: best = i best_distance = distance @@ -82,19 +78,17 @@ def predict(self, x): class KMeans(object): @classmethod - def train(cls, data, k, maxIterations=100, runs=1, initializationMode="k-means||"): + def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||"): """Train a k-means clustering model.""" - sc = data.context - dataBytes = _get_unmangled_double_vector_rdd(data) - ans = sc._jvm.PythonMLLibAPI().trainKMeansModel( - dataBytes._jrdd, k, maxIterations, runs, initializationMode) - if len(ans) != 1: - raise RuntimeError("JVM call result had unexpected length") - elif type(ans[0]) != bytearray: - raise RuntimeError("JVM call result had first element of type " - + type(ans[0]) + " which is not bytearray") - matrix = _deserialize_double_matrix(ans[0]) - return KMeansModel([row for row in matrix]) + sc = rdd.context + ser = PickleSerializer() + # cache serialized data to avoid objects over head in JVM + cached = rdd.map(_convert_to_vector)._reserialize(AutoBatchedSerializer(ser)).cache() + model = sc._jvm.PythonMLLibAPI().trainKMeansModel( + cached._to_java_object_rdd(), k, maxIterations, runs, initializationMode) + bytes = sc._jvm.SerDe.dumps(model.clusterCenters()) + centers = ser.loads(str(bytes)) + return KMeansModel([c.toArray() for c in centers]) def _test(): diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index e69051c104e37..0a5dcaac55e46 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -23,14 +23,148 @@ SciPy is available in their environment. """ -import numpy -from numpy import array, array_equal, ndarray, float64, int32 +import sys +import array +import copy_reg +import numpy as np -__all__ = ['SparseVector', 'Vectors'] +__all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors'] -class SparseVector(object): +if sys.version_info[:2] == (2, 7): + # speed up pickling array in Python 2.7 + def fast_pickle_array(ar): + return array.array, (ar.typecode, ar.tostring()) + copy_reg.pickle(array.array, fast_pickle_array) + + +# Check whether we have SciPy. MLlib works without it too, but if we have it, some methods, +# such as _dot and _serialize_double_vector, start to support scipy.sparse matrices. + +try: + import scipy.sparse + _have_scipy = True +except: + # No SciPy in environment, but that's okay + _have_scipy = False + + +def _convert_to_vector(l): + if isinstance(l, Vector): + return l + elif type(l) in (array.array, np.array, np.ndarray, list, tuple): + return DenseVector(l) + elif _have_scipy and scipy.sparse.issparse(l): + assert l.shape[1] == 1, "Expected column vector" + csc = l.tocsc() + return SparseVector(l.shape[0], csc.indices, csc.data) + else: + raise TypeError("Cannot convert type %s into Vector" % type(l)) + + +class Vector(object): + """ + Abstract class for DenseVector and SparseVector + """ + def toArray(self): + """ + Convert the vector into an numpy.ndarray + :return: numpy.ndarray + """ + raise NotImplementedError + + +class DenseVector(Vector): + def __init__(self, ar): + if not isinstance(ar, array.array): + ar = array.array('d', ar) + self.array = ar + + def __reduce__(self): + return DenseVector, (self.array,) + + def dot(self, other): + """ + Compute the dot product of two Vectors. We support + (Numpy array, list, SparseVector, or SciPy sparse) + and a target NumPy array that is either 1- or 2-dimensional. + Equivalent to calling numpy.dot of the two vectors. + + >>> dense = DenseVector(array.array('d', [1., 2.])) + >>> dense.dot(dense) + 5.0 + >>> dense.dot(SparseVector(2, [0, 1], [2., 1.])) + 4.0 + >>> dense.dot(range(1, 3)) + 5.0 + >>> dense.dot(np.array(range(1, 3))) + 5.0 + """ + if isinstance(other, SparseVector): + return other.dot(self) + elif _have_scipy and scipy.sparse.issparse(other): + return other.transpose().dot(self.toArray())[0] + elif isinstance(other, Vector): + return np.dot(self.toArray(), other.toArray()) + else: + return np.dot(self.toArray(), other) + + def squared_distance(self, other): + """ + Squared distance of two Vectors. + + >>> dense1 = DenseVector(array.array('d', [1., 2.])) + >>> dense1.squared_distance(dense1) + 0.0 + >>> dense2 = np.array([2., 1.]) + >>> dense1.squared_distance(dense2) + 2.0 + >>> dense3 = [2., 1.] + >>> dense1.squared_distance(dense3) + 2.0 + >>> sparse1 = SparseVector(2, [0, 1], [2., 1.]) + >>> dense1.squared_distance(sparse1) + 2.0 + """ + if isinstance(other, SparseVector): + return other.squared_distance(self) + elif _have_scipy and scipy.sparse.issparse(other): + return _convert_to_vector(other).squared_distance(self) + + if isinstance(other, Vector): + other = other.toArray() + elif not isinstance(other, np.ndarray): + other = np.array(other) + diff = self.toArray() - other + return np.dot(diff, diff) + + def toArray(self): + return np.array(self.array) + + def __getitem__(self, item): + return self.array[item] + + def __len__(self): + return len(self.array) + + def __str__(self): + return "[" + ",".join([str(v) for v in self.array]) + "]" + + def __repr__(self): + return "DenseVector(%r)" % self.array + + def __eq__(self, other): + return isinstance(other, DenseVector) and self.array == other.array + + def __ne__(self, other): + return not self == other + + def __getattr__(self, item): + return getattr(self.array, item) + + +class SparseVector(Vector): """ A simple sparse vector class for passing data to MLlib. Users may @@ -61,16 +195,19 @@ def __init__(self, size, *args): if type(pairs) == dict: pairs = pairs.items() pairs = sorted(pairs) - self.indices = array([p[0] for p in pairs], dtype=int32) - self.values = array([p[1] for p in pairs], dtype=float64) + self.indices = array.array('i', [p[0] for p in pairs]) + self.values = array.array('d', [p[1] for p in pairs]) else: assert len(args[0]) == len(args[1]), "index and value arrays not same length" - self.indices = array(args[0], dtype=int32) - self.values = array(args[1], dtype=float64) + self.indices = array.array('i', args[0]) + self.values = array.array('d', args[1]) for i in xrange(len(self.indices) - 1): if self.indices[i] >= self.indices[i + 1]: raise TypeError("indices array must be sorted") + def __reduce__(self): + return (SparseVector, (self.size, self.indices, self.values)) + def dot(self, other): """ Dot product with a SparseVector or 1- or 2-dimensional Numpy array. @@ -78,15 +215,15 @@ def dot(self, other): >>> a = SparseVector(4, [1, 3], [3.0, 4.0]) >>> a.dot(a) 25.0 - >>> a.dot(array([1., 2., 3., 4.])) + >>> a.dot(array.array('d', [1., 2., 3., 4.])) 22.0 >>> b = SparseVector(4, [2, 4], [1.0, 2.0]) >>> a.dot(b) 0.0 - >>> a.dot(array([[1, 1], [2, 2], [3, 3], [4, 4]])) + >>> a.dot(np.array([[1, 1], [2, 2], [3, 3], [4, 4]])) array([ 22., 22.]) """ - if type(other) == ndarray: + if type(other) == np.ndarray: if other.ndim == 1: result = 0.0 for i in xrange(len(self.indices)): @@ -94,10 +231,17 @@ def dot(self, other): return result elif other.ndim == 2: results = [self.dot(other[:, i]) for i in xrange(other.shape[1])] - return array(results) + return np.array(results) else: raise Exception("Cannot call dot with %d-dimensional array" % other.ndim) - else: + + elif type(other) in (array.array, DenseVector): + result = 0.0 + for i in xrange(len(self.indices)): + result += self.values[i] * other[self.indices[i]] + return result + + elif type(other) is SparseVector: result = 0.0 i, j = 0, 0 while i < len(self.indices) and j < len(other.indices): @@ -110,6 +254,8 @@ def dot(self, other): else: j += 1 return result + else: + return self.dot(_convert_to_vector(other)) def squared_distance(self, other): """ @@ -118,7 +264,9 @@ def squared_distance(self, other): >>> a = SparseVector(4, [1, 3], [3.0, 4.0]) >>> a.squared_distance(a) 0.0 - >>> a.squared_distance(array([1., 2., 3., 4.])) + >>> a.squared_distance(array.array('d', [1., 2., 3., 4.])) + 11.0 + >>> a.squared_distance(np.array([1., 2., 3., 4.])) 11.0 >>> b = SparseVector(4, [2, 4], [1.0, 2.0]) >>> a.squared_distance(b) @@ -126,22 +274,22 @@ def squared_distance(self, other): >>> b.squared_distance(a) 30.0 """ - if type(other) == ndarray: - if other.ndim == 1: - result = 0.0 - j = 0 # index into our own array - for i in xrange(other.shape[0]): - if j < len(self.indices) and self.indices[j] == i: - diff = self.values[j] - other[i] - result += diff * diff - j += 1 - else: - result += other[i] * other[i] - return result - else: + if type(other) in (list, array.array, DenseVector, np.array, np.ndarray): + if type(other) is np.array and other.ndim != 1: raise Exception("Cannot call squared_distance with %d-dimensional array" % other.ndim) - else: + result = 0.0 + j = 0 # index into our own array + for i in xrange(len(other)): + if j < len(self.indices) and self.indices[j] == i: + diff = self.values[j] - other[i] + result += diff * diff + j += 1 + else: + result += other[i] * other[i] + return result + + elif type(other) is SparseVector: result = 0.0 i, j = 0, 0 while i < len(self.indices) and j < len(other.indices): @@ -163,16 +311,21 @@ def squared_distance(self, other): result += other.values[j] * other.values[j] j += 1 return result + else: + return self.squared_distance(_convert_to_vector(other)) def toArray(self): """ Returns a copy of this SparseVector as a 1-dimensional NumPy array. """ - arr = numpy.zeros(self.size) + arr = np.zeros((self.size,), dtype=np.float64) for i in xrange(self.indices.size): arr[self.indices[i]] = self.values[i] return arr + def __len__(self): + return self.size + def __str__(self): inds = "[" + ",".join([str(i) for i in self.indices]) + "]" vals = "[" + ",".join([str(v) for v in self.values]) + "]" @@ -198,8 +351,8 @@ def __eq__(self, other): return (isinstance(other, self.__class__) and other.size == self.size - and array_equal(other.indices, self.indices) - and array_equal(other.values, self.values)) + and other.indices == self.indices + and other.values == self.values) def __ne__(self, other): return not self.__eq__(other) @@ -242,9 +395,9 @@ def dense(elements): returns a NumPy array. >>> Vectors.dense([1, 2, 3]) - array([ 1., 2., 3.]) + DenseVector(array('d', [1.0, 2.0, 3.0])) """ - return array(elements, dtype=float64) + return DenseVector(elements) @staticmethod def stringify(vector): @@ -257,10 +410,39 @@ def stringify(vector): >>> Vectors.stringify(Vectors.dense([0.0, 1.0])) '[0.0,1.0]' """ - if type(vector) == SparseVector: - return str(vector) - else: - return "[" + ",".join([str(v) for v in vector]) + "]" + return str(vector) + + +class Matrix(object): + """ the Matrix """ + def __init__(self, nRow, nCol): + self.nRow = nRow + self.nCol = nCol + + def toArray(self): + raise NotImplementedError + + +class DenseMatrix(Matrix): + def __init__(self, nRow, nCol, values): + Matrix.__init__(self, nRow, nCol) + assert len(values) == nRow * nCol + self.values = values + + def __reduce__(self): + return DenseMatrix, (self.nRow, self.nCol, self.values) + + def toArray(self): + """ + Return an numpy.ndarray + + >>> arr = array.array('d', [float(i) for i in range(4)]) + >>> m = DenseMatrix(2, 2, arr) + >>> m.toArray() + array([[ 0., 1.], + [ 2., 3.]]) + """ + return np.ndarray((self.nRow, self.nCol), np.float64, buffer=self.values.tostring()) def _test(): diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index d53c95fd59c25..a787e4dea2c55 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -19,15 +19,32 @@ Python package for random data generation. """ +from functools import wraps from pyspark.rdd import RDD -from pyspark.mllib._common import _deserialize_double, _deserialize_double_vector -from pyspark.serializers import NoOpSerializer +from pyspark.serializers import BatchedSerializer, PickleSerializer __all__ = ['RandomRDDs', ] +def serialize(f): + @wraps(f) + def func(sc, *a, **kw): + jrdd = f(sc, *a, **kw) + return RDD(sc._jvm.PythonRDD.javaToPython(jrdd), sc, + BatchedSerializer(PickleSerializer(), 1024)) + return func + + +def toArray(f): + @wraps(f) + def func(sc, *a, **kw): + rdd = f(sc, *a, **kw) + return rdd.map(lambda vec: vec.toArray()) + return func + + class RandomRDDs(object): """ Generator methods for creating RDDs comprised of i.i.d samples from @@ -35,6 +52,7 @@ class RandomRDDs(object): """ @staticmethod + @serialize def uniformRDD(sc, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the @@ -56,11 +74,10 @@ def uniformRDD(sc, size, numPartitions=None, seed=None): >>> parts == sc.defaultParallelism True """ - jrdd = sc._jvm.PythonMLLibAPI().uniformRDD(sc._jsc, size, numPartitions, seed) - uniform = RDD(jrdd, sc, NoOpSerializer()) - return uniform.map(lambda bytes: _deserialize_double(bytearray(bytes))) + return sc._jvm.PythonMLLibAPI().uniformRDD(sc._jsc, size, numPartitions, seed) @staticmethod + @serialize def normalRDD(sc, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the standard normal @@ -80,11 +97,10 @@ def normalRDD(sc, size, numPartitions=None, seed=None): >>> abs(stats.stdev() - 1.0) < 0.1 True """ - jrdd = sc._jvm.PythonMLLibAPI().normalRDD(sc._jsc, size, numPartitions, seed) - normal = RDD(jrdd, sc, NoOpSerializer()) - return normal.map(lambda bytes: _deserialize_double(bytearray(bytes))) + return sc._jvm.PythonMLLibAPI().normalRDD(sc._jsc, size, numPartitions, seed) @staticmethod + @serialize def poissonRDD(sc, mean, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the Poisson @@ -101,11 +117,11 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): >>> abs(stats.stdev() - sqrt(mean)) < 0.5 True """ - jrdd = sc._jvm.PythonMLLibAPI().poissonRDD(sc._jsc, mean, size, numPartitions, seed) - poisson = RDD(jrdd, sc, NoOpSerializer()) - return poisson.map(lambda bytes: _deserialize_double(bytearray(bytes))) + return sc._jvm.PythonMLLibAPI().poissonRDD(sc._jsc, mean, size, numPartitions, seed) @staticmethod + @toArray + @serialize def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): """ Generates an RDD comprised of vectors containing i.i.d. samples drawn @@ -120,12 +136,12 @@ def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): >>> RandomRDDs.uniformVectorRDD(sc, 10, 10, 4).getNumPartitions() 4 """ - jrdd = sc._jvm.PythonMLLibAPI() \ + return sc._jvm.PythonMLLibAPI() \ .uniformVectorRDD(sc._jsc, numRows, numCols, numPartitions, seed) - uniform = RDD(jrdd, sc, NoOpSerializer()) - return uniform.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) @staticmethod + @toArray + @serialize def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): """ Generates an RDD comprised of vectors containing i.i.d. samples drawn @@ -140,12 +156,12 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): >>> abs(mat.std() - 1.0) < 0.1 True """ - jrdd = sc._jvm.PythonMLLibAPI() \ + return sc._jvm.PythonMLLibAPI() \ .normalVectorRDD(sc._jsc, numRows, numCols, numPartitions, seed) - normal = RDD(jrdd, sc, NoOpSerializer()) - return normal.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) @staticmethod + @toArray + @serialize def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): """ Generates an RDD comprised of vectors containing i.i.d. samples drawn @@ -163,10 +179,8 @@ def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): >>> abs(mat.std() - sqrt(mean)) < 0.5 True """ - jrdd = sc._jvm.PythonMLLibAPI() \ + return sc._jvm.PythonMLLibAPI() \ .poissonVectorRDD(sc._jsc, mean, numRows, numCols, numPartitions, seed) - poisson = RDD(jrdd, sc, NoOpSerializer()) - return poisson.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) def _test(): diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 2df23394da6f8..59c1c5ff0ced0 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -16,17 +16,25 @@ # from pyspark import SparkContext -from pyspark.mllib._common import \ - _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \ - _serialize_double_matrix, _deserialize_double_matrix, \ - _serialize_double_vector, _deserialize_double_vector, \ - _get_initial_weights, _serialize_rating, _regression_train_wrapper, \ - _serialize_tuple, RatingDeserializer +from pyspark.serializers import PickleSerializer, AutoBatchedSerializer from pyspark.rdd import RDD __all__ = ['MatrixFactorizationModel', 'ALS'] +class Rating(object): + def __init__(self, user, product, rating): + self.user = int(user) + self.product = int(product) + self.rating = float(rating) + + def __reduce__(self): + return Rating, (self.user, self.product, self.rating) + + def __repr__(self): + return "Rating(%d, %d, %d)" % (self.user, self.product, self.rating) + + class MatrixFactorizationModel(object): """A matrix factorisation model trained by regularized alternating @@ -39,7 +47,9 @@ class MatrixFactorizationModel(object): >>> model = ALS.trainImplicit(ratings, 1) >>> model.predict(2,2) is not None True + >>> testset = sc.parallelize([(1, 2), (1, 1)]) + >>> model = ALS.train(ratings, 1) >>> model.predictAll(testset).count() == 2 True """ @@ -54,34 +64,61 @@ def __del__(self): def predict(self, user, product): return self._java_model.predict(user, product) - def predictAll(self, usersProducts): - usersProductsJRDD = _get_unmangled_rdd(usersProducts, _serialize_tuple) - return RDD(self._java_model.predict(usersProductsJRDD._jrdd), - self._context, RatingDeserializer()) + def predictAll(self, user_product): + assert isinstance(user_product, RDD), "user_product should be RDD of (user, product)" + first = user_product.first() + if isinstance(first, list): + user_product = user_product.map(tuple) + first = tuple(first) + assert type(first) is tuple and len(first) == 2, \ + "user_product should be RDD of (user, product)" + if any(isinstance(x, str) for x in first): + user_product = user_product.map(lambda (u, p): (int(x), int(p))) + first = tuple(map(int, first)) + assert all(type(x) is int for x in first), "user and product in user_product shoul be int" + sc = self._context + tuplerdd = sc._jvm.SerDe.asTupleRDD(user_product._to_java_object_rdd().rdd()) + jresult = self._java_model.predict(tuplerdd).toJavaRDD() + return RDD(sc._jvm.PythonRDD.javaToPython(jresult), sc, + AutoBatchedSerializer(PickleSerializer())) class ALS(object): + @classmethod + def _prepare(cls, ratings): + assert isinstance(ratings, RDD), "ratings should be RDD" + first = ratings.first() + if not isinstance(first, Rating): + if isinstance(first, (tuple, list)): + ratings = ratings.map(lambda x: Rating(*x)) + else: + raise ValueError("rating should be RDD of Rating or tuple/list") + # serialize them by AutoBatchedSerializer before cache to reduce the + # objects overhead in JVM + cached = ratings._reserialize(AutoBatchedSerializer(PickleSerializer())).cache() + return cached._to_java_object_rdd() + @classmethod def train(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1): sc = ratings.context - ratingBytes = _get_unmangled_rdd(ratings, _serialize_rating) - mod = sc._jvm.PythonMLLibAPI().trainALSModel( - ratingBytes._jrdd, rank, iterations, lambda_, blocks) + jrating = cls._prepare(ratings) + mod = sc._jvm.PythonMLLibAPI().trainALSModel(jrating, rank, iterations, lambda_, blocks) return MatrixFactorizationModel(sc, mod) @classmethod def trainImplicit(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1, alpha=0.01): sc = ratings.context - ratingBytes = _get_unmangled_rdd(ratings, _serialize_rating) + jrating = cls._prepare(ratings) mod = sc._jvm.PythonMLLibAPI().trainImplicitALSModel( - ratingBytes._jrdd, rank, iterations, lambda_, blocks, alpha) + jrating, rank, iterations, lambda_, blocks, alpha) return MatrixFactorizationModel(sc, mod) def _test(): import doctest - globs = globals().copy() + import pyspark.mllib.recommendation + globs = pyspark.mllib.recommendation.__dict__.copy() globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index f572dcfb840b6..cbdbc09858013 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -15,12 +15,12 @@ # limitations under the License. # -from numpy import array, ndarray -from pyspark import SparkContext -from pyspark.mllib._common import _dot, _regression_train_wrapper, \ - _linear_predictor_typecheck, _have_scipy, _scipy_issparse -from pyspark.mllib.linalg import SparseVector, Vectors +import numpy as np +from numpy import array +from pyspark import SparkContext +from pyspark.mllib.linalg import SparseVector, _convert_to_vector +from pyspark.serializers import PickleSerializer, AutoBatchedSerializer __all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'RidgeRegressionModel' 'LinearRegressionWithSGD', 'LassoWithSGD', 'RidgeRegressionWithSGD'] @@ -38,16 +38,16 @@ class LabeledPoint(object): def __init__(self, label, features): self.label = label - if (type(features) == ndarray or type(features) == SparseVector - or (_have_scipy and _scipy_issparse(features))): - self.features = features - elif type(features) == list: - self.features = array(features) - else: - raise TypeError("Expected NumPy array, list, SparseVector, or scipy.sparse matrix") + self.features = _convert_to_vector(features) + + def __reduce__(self): + return (LabeledPoint, (self.label, self.features)) def __str__(self): - return "(" + ",".join((str(self.label), Vectors.stringify(self.features))) + ")" + return "(" + ",".join((str(self.label), str(self.features))) + ")" + + def __repr__(self): + return "LabeledPoint(" + ",".join((repr(self.label), repr(self.features))) + ")" class LinearModel(object): @@ -55,7 +55,7 @@ class LinearModel(object): """A linear model that has a vector of coefficients and an intercept.""" def __init__(self, weights, intercept): - self._coeff = weights + self._coeff = _convert_to_vector(weights) self._intercept = intercept @property @@ -71,18 +71,19 @@ class LinearRegressionModelBase(LinearModel): """A linear regression model. - >>> lrmb = LinearRegressionModelBase(array([1.0, 2.0]), 0.1) - >>> abs(lrmb.predict(array([-1.03, 7.777])) - 14.624) < 1e-6 + >>> lrmb = LinearRegressionModelBase(np.array([1.0, 2.0]), 0.1) + >>> abs(lrmb.predict(np.array([-1.03, 7.777])) - 14.624) < 1e-6 True >>> abs(lrmb.predict(SparseVector(2, {0: -1.03, 1: 7.777})) - 14.624) < 1e-6 True """ def predict(self, x): - """Predict the value of the dependent variable given a vector x""" - """containing values for the independent variables.""" - _linear_predictor_typecheck(x, self._coeff) - return _dot(x, self._coeff) + self._intercept + """ + Predict the value of the dependent variable given a vector x + containing values for the independent variables. + """ + return self.weights.dot(x) + self.intercept class LinearRegressionModel(LinearRegressionModelBase): @@ -96,10 +97,10 @@ class LinearRegressionModel(LinearRegressionModelBase): ... LabeledPoint(3.0, [2.0]), ... LabeledPoint(2.0, [3.0]) ... ] - >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=np.array([1.0])) + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True - >>> abs(lrm.predict(array([1.0])) - 1) < 0.5 + >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True @@ -117,11 +118,27 @@ class LinearRegressionModel(LinearRegressionModelBase): """ +# train_func should take two parameters, namely data and initial_weights, and +# return the result of a call to the appropriate JVM stub. +# _regression_train_wrapper is responsible for setup and error checking. +def _regression_train_wrapper(sc, train_func, modelClass, data, initial_weights): + initial_weights = initial_weights or [0.0] * len(data.first().features) + ser = PickleSerializer() + initial_bytes = bytearray(ser.dumps(_convert_to_vector(initial_weights))) + # use AutoBatchedSerializer before cache to reduce the memory + # overhead in JVM + cached = data._reserialize(AutoBatchedSerializer(ser)).cache() + ans = train_func(cached._to_java_object_rdd(), initial_bytes) + assert len(ans) == 2, "JVM call result had unexpected length" + weights = ser.loads(str(ans[0])) + return modelClass(weights, ans[1]) + + class LinearRegressionWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, - initialWeights=None, regParam=1.0, regType=None, intercept=False): + initialWeights=None, regParam=1.0, regType="none", intercept=False): """ Train a linear regression model on the given data. @@ -146,11 +163,12 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, are activated or not). """ sc = data.context - if regType is None: - regType = "none" - train_f = lambda d, i: sc._jvm.PythonMLLibAPI().trainLinearRegressionModelWithSGD( - d._jrdd, iterations, step, miniBatchFraction, i, regParam, regType, intercept) - return _regression_train_wrapper(sc, train_f, LinearRegressionModel, data, initialWeights) + + def train(jrdd, i): + return sc._jvm.PythonMLLibAPI().trainLinearRegressionModelWithSGD( + jrdd, iterations, step, miniBatchFraction, i, regParam, regType, intercept) + + return _regression_train_wrapper(sc, train, LinearRegressionModel, data, initialWeights) class LassoModel(LinearRegressionModelBase): @@ -166,9 +184,9 @@ class LassoModel(LinearRegressionModelBase): ... LabeledPoint(2.0, [3.0]) ... ] >>> lrm = LassoWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True - >>> abs(lrm.predict(array([1.0])) - 1) < 0.5 + >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True @@ -179,7 +197,7 @@ class LassoModel(LinearRegressionModelBase): ... LabeledPoint(2.0, SparseVector(1, {0: 3.0})) ... ] >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True @@ -193,9 +211,11 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): """Train a Lasso regression model on the given data.""" sc = data.context - train_f = lambda d, i: sc._jvm.PythonMLLibAPI().trainLassoModelWithSGD( - d._jrdd, iterations, step, regParam, miniBatchFraction, i) - return _regression_train_wrapper(sc, train_f, LassoModel, data, initialWeights) + + def train(jrdd, i): + return sc._jvm.PythonMLLibAPI().trainLassoModelWithSGD( + jrdd, iterations, step, regParam, miniBatchFraction, i) + return _regression_train_wrapper(sc, train, LassoModel, data, initialWeights) class RidgeRegressionModel(LinearRegressionModelBase): @@ -211,9 +231,9 @@ class RidgeRegressionModel(LinearRegressionModelBase): ... LabeledPoint(2.0, [3.0]) ... ] >>> lrm = RidgeRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True - >>> abs(lrm.predict(array([1.0])) - 1) < 0.5 + >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True @@ -224,7 +244,7 @@ class RidgeRegressionModel(LinearRegressionModelBase): ... LabeledPoint(2.0, SparseVector(1, {0: 3.0})) ... ] >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True @@ -238,9 +258,12 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): """Train a ridge regression model on the given data.""" sc = data.context - train_func = lambda d, i: sc._jvm.PythonMLLibAPI().trainRidgeModelWithSGD( - d._jrdd, iterations, step, regParam, miniBatchFraction, i) - return _regression_train_wrapper(sc, train_func, RidgeRegressionModel, data, initialWeights) + + def train(jrdd, i): + return sc._jvm.PythonMLLibAPI().trainRidgeModelWithSGD( + jrdd, iterations, step, regParam, miniBatchFraction, i) + + return _regression_train_wrapper(sc, train, RidgeRegressionModel, data, initialWeights) def _test(): diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index 8c726f171c978..b9de0909a6fb1 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -19,14 +19,26 @@ Python package for statistical functions in MLlib. """ -from pyspark.mllib._common import \ - _get_unmangled_double_vector_rdd, _get_unmangled_rdd, \ - _serialize_double, _deserialize_double_matrix, _deserialize_double_vector +from functools import wraps + +from pyspark import PickleSerializer __all__ = ['MultivariateStatisticalSummary', 'Statistics'] +def serialize(f): + ser = PickleSerializer() + + @wraps(f) + def func(self): + jvec = f(self) + bytes = self._sc._jvm.SerDe.dumps(jvec) + return ser.loads(str(bytes)).toArray() + + return func + + class MultivariateStatisticalSummary(object): """ @@ -44,33 +56,38 @@ def __init__(self, sc, java_summary): def __del__(self): self._sc._gateway.detach(self._java_summary) + @serialize def mean(self): - return _deserialize_double_vector(self._java_summary.mean()) + return self._java_summary.mean() + @serialize def variance(self): - return _deserialize_double_vector(self._java_summary.variance()) + return self._java_summary.variance() def count(self): return self._java_summary.count() + @serialize def numNonzeros(self): - return _deserialize_double_vector(self._java_summary.numNonzeros()) + return self._java_summary.numNonzeros() + @serialize def max(self): - return _deserialize_double_vector(self._java_summary.max()) + return self._java_summary.max() + @serialize def min(self): - return _deserialize_double_vector(self._java_summary.min()) + return self._java_summary.min() class Statistics(object): @staticmethod - def colStats(X): + def colStats(rdd): """ Computes column-wise summary statistics for the input RDD[Vector]. - >>> from linalg import Vectors + >>> from pyspark.mllib.linalg import Vectors >>> rdd = sc.parallelize([Vectors.dense([2, 0, 0, -2]), ... Vectors.dense([4, 5, 0, 3]), ... Vectors.dense([6, 7, 0, 8])]) @@ -88,9 +105,9 @@ def colStats(X): >>> cStats.min() array([ 2., 0., 0., -2.]) """ - sc = X.ctx - Xser = _get_unmangled_double_vector_rdd(X) - cStats = sc._jvm.PythonMLLibAPI().colStats(Xser._jrdd) + sc = rdd.ctx + jrdd = rdd._to_java_object_rdd() + cStats = sc._jvm.PythonMLLibAPI().colStats(jrdd) return MultivariateStatisticalSummary(sc, cStats) @staticmethod @@ -117,7 +134,7 @@ def corr(x, y=None, method=None): >>> from math import isnan >>> isnan(Statistics.corr(x, zeros)) True - >>> from linalg import Vectors + >>> from pyspark.mllib.linalg import Vectors >>> rdd = sc.parallelize([Vectors.dense([1, 0, 0, -2]), Vectors.dense([4, 5, 0, 3]), ... Vectors.dense([6, 7, 0, 8]), Vectors.dense([9, 0, 0, 1])]) >>> pearsonCorr = Statistics.corr(rdd) @@ -144,18 +161,16 @@ def corr(x, y=None, method=None): # check if y is used to specify the method name instead. if type(y) == str: raise TypeError("Use 'method=' to specify method name.") + + jx = x._to_java_object_rdd() if not y: - try: - Xser = _get_unmangled_double_vector_rdd(x) - except TypeError: - raise TypeError("corr called on a single RDD not consisted of Vectors.") - resultMat = sc._jvm.PythonMLLibAPI().corr(Xser._jrdd, method) - return _deserialize_double_matrix(resultMat) + resultMat = sc._jvm.PythonMLLibAPI().corr(jx, method) + bytes = sc._jvm.SerDe.dumps(resultMat) + ser = PickleSerializer() + return ser.loads(str(bytes)).toArray() else: - xSer = _get_unmangled_rdd(x, _serialize_double) - ySer = _get_unmangled_rdd(y, _serialize_double) - result = sc._jvm.PythonMLLibAPI().corr(xSer._jrdd, ySer._jrdd, method) - return result + jy = y._to_java_object_rdd() + return sc._jvm.PythonMLLibAPI().corr(jx, jy, method) def _test(): diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 8a851bd35c0e8..f72e88ba6e2ba 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -20,6 +20,8 @@ """ import sys +import array as pyarray + from numpy import array, array_equal if sys.version_info[:2] <= (2, 6): @@ -27,9 +29,8 @@ else: import unittest -from pyspark.mllib._common import _convert_vector, _serialize_double_vector, \ - _deserialize_double_vector, _dot, _squared_distance -from pyspark.mllib.linalg import SparseVector +from pyspark.serializers import PickleSerializer +from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint from pyspark.tests import PySparkTestCase @@ -42,39 +43,52 @@ # No SciPy, but that's okay, we'll skip those tests pass +ser = PickleSerializer() + + +def _squared_distance(a, b): + if isinstance(a, Vector): + return a.squared_distance(b) + else: + return b.squared_distance(a) -class VectorTests(unittest.TestCase): + +class VectorTests(PySparkTestCase): + + def _test_serialize(self, v): + jvec = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(v))) + nv = ser.loads(str(self.sc._jvm.SerDe.dumps(jvec))) + self.assertEqual(v, nv) + vs = [v] * 100 + jvecs = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(vs))) + nvs = ser.loads(str(self.sc._jvm.SerDe.dumps(jvecs))) + self.assertEqual(vs, nvs) def test_serialize(self): - sv = SparseVector(4, {1: 1, 3: 2}) - dv = array([1., 2., 3., 4.]) - lst = [1, 2, 3, 4] - self.assertTrue(sv is _convert_vector(sv)) - self.assertTrue(dv is _convert_vector(dv)) - self.assertTrue(array_equal(dv, _convert_vector(lst))) - self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(sv))) - self.assertTrue(array_equal(dv, _deserialize_double_vector(_serialize_double_vector(dv)))) - self.assertTrue(array_equal(dv, _deserialize_double_vector(_serialize_double_vector(lst)))) + self._test_serialize(DenseVector(range(10))) + self._test_serialize(DenseVector(array([1., 2., 3., 4.]))) + self._test_serialize(DenseVector(pyarray.array('d', range(10)))) + self._test_serialize(SparseVector(4, {1: 1, 3: 2})) def test_dot(self): sv = SparseVector(4, {1: 1, 3: 2}) - dv = array([1., 2., 3., 4.]) - lst = [1, 2, 3, 4] + dv = DenseVector(array([1., 2., 3., 4.])) + lst = DenseVector([1, 2, 3, 4]) mat = array([[1., 2., 3., 4.], [1., 2., 3., 4.], [1., 2., 3., 4.], [1., 2., 3., 4.]]) - self.assertEquals(10.0, _dot(sv, dv)) - self.assertTrue(array_equal(array([3., 6., 9., 12.]), _dot(sv, mat))) - self.assertEquals(30.0, _dot(dv, dv)) - self.assertTrue(array_equal(array([10., 20., 30., 40.]), _dot(dv, mat))) - self.assertEquals(30.0, _dot(lst, dv)) - self.assertTrue(array_equal(array([10., 20., 30., 40.]), _dot(lst, mat))) + self.assertEquals(10.0, sv.dot(dv)) + self.assertTrue(array_equal(array([3., 6., 9., 12.]), sv.dot(mat))) + self.assertEquals(30.0, dv.dot(dv)) + self.assertTrue(array_equal(array([10., 20., 30., 40.]), dv.dot(mat))) + self.assertEquals(30.0, lst.dot(dv)) + self.assertTrue(array_equal(array([10., 20., 30., 40.]), lst.dot(mat))) def test_squared_distance(self): sv = SparseVector(4, {1: 1, 3: 2}) - dv = array([1., 2., 3., 4.]) - lst = [4, 3, 2, 1] + dv = DenseVector(array([1., 2., 3., 4.])) + lst = DenseVector([4, 3, 2, 1]) self.assertEquals(15.0, _squared_distance(sv, dv)) self.assertEquals(25.0, _squared_distance(sv, lst)) self.assertEquals(20.0, _squared_distance(dv, lst)) @@ -198,41 +212,36 @@ def test_serialize(self): lil[1, 0] = 1 lil[3, 0] = 2 sv = SparseVector(4, {1: 1, 3: 2}) - self.assertEquals(sv, _convert_vector(lil)) - self.assertEquals(sv, _convert_vector(lil.tocsc())) - self.assertEquals(sv, _convert_vector(lil.tocoo())) - self.assertEquals(sv, _convert_vector(lil.tocsr())) - self.assertEquals(sv, _convert_vector(lil.todok())) - self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(lil))) - self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(lil.tocsc()))) - self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(lil.tocsr()))) - self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(lil.todok()))) + self.assertEquals(sv, _convert_to_vector(lil)) + self.assertEquals(sv, _convert_to_vector(lil.tocsc())) + self.assertEquals(sv, _convert_to_vector(lil.tocoo())) + self.assertEquals(sv, _convert_to_vector(lil.tocsr())) + self.assertEquals(sv, _convert_to_vector(lil.todok())) + + def serialize(l): + return ser.loads(ser.dumps(_convert_to_vector(l))) + self.assertEquals(sv, serialize(lil)) + self.assertEquals(sv, serialize(lil.tocsc())) + self.assertEquals(sv, serialize(lil.tocsr())) + self.assertEquals(sv, serialize(lil.todok())) def test_dot(self): from scipy.sparse import lil_matrix lil = lil_matrix((4, 1)) lil[1, 0] = 1 lil[3, 0] = 2 - dv = array([1., 2., 3., 4.]) - sv = SparseVector(4, {0: 1, 1: 2, 2: 3, 3: 4}) - mat = array([[1., 2., 3., 4.], - [1., 2., 3., 4.], - [1., 2., 3., 4.], - [1., 2., 3., 4.]]) - self.assertEquals(10.0, _dot(lil, dv)) - self.assertTrue(array_equal(array([3., 6., 9., 12.]), _dot(lil, mat))) + dv = DenseVector(array([1., 2., 3., 4.])) + self.assertEquals(10.0, dv.dot(lil)) def test_squared_distance(self): from scipy.sparse import lil_matrix lil = lil_matrix((4, 1)) lil[1, 0] = 3 lil[3, 0] = 2 - dv = array([1., 2., 3., 4.]) + dv = DenseVector(array([1., 2., 3., 4.])) sv = SparseVector(4, {0: 1, 1: 2, 2: 3, 3: 4}) - self.assertEquals(15.0, _squared_distance(lil, dv)) - self.assertEquals(15.0, _squared_distance(lil, sv)) - self.assertEquals(15.0, _squared_distance(dv, lil)) - self.assertEquals(15.0, _squared_distance(sv, lil)) + self.assertEquals(15.0, dv.squared_distance(lil)) + self.assertEquals(15.0, sv.squared_distance(lil)) def scipy_matrix(self, size, values): """Create a column SciPy matrix from a dictionary of values""" diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 5b13ab682bbfc..f59a818a6e74d 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -18,13 +18,9 @@ from py4j.java_collections import MapConverter from pyspark import SparkContext, RDD -from pyspark.mllib._common import \ - _get_unmangled_rdd, _get_unmangled_double_vector_rdd, _serialize_double_vector, \ - _deserialize_labeled_point, _get_unmangled_labeled_point_rdd, \ - _deserialize_double +from pyspark.serializers import BatchedSerializer, PickleSerializer +from pyspark.mllib.linalg import Vector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint -from pyspark.serializers import NoOpSerializer - __all__ = ['DecisionTreeModel', 'DecisionTree'] @@ -55,21 +51,24 @@ def predict(self, x): :param x: Data point (feature vector), or an RDD of data points (feature vectors). """ - pythonAPI = self._sc._jvm.PythonMLLibAPI() + SerDe = self._sc._jvm.SerDe + ser = PickleSerializer() if isinstance(x, RDD): # Bulk prediction - if x.count() == 0: + first = x.take(1) + if not first: return self._sc.parallelize([]) - dataBytes = _get_unmangled_double_vector_rdd(x, cache=False) - jSerializedPreds = \ - pythonAPI.predictDecisionTreeModel(self._java_model, - dataBytes._jrdd) - serializedPreds = RDD(jSerializedPreds, self._sc, NoOpSerializer()) - return serializedPreds.map(lambda bytes: _deserialize_double(bytearray(bytes))) + if not isinstance(first[0], Vector): + x = x.map(_convert_to_vector) + jPred = self._java_model.predict(x._to_java_object_rdd()).toJavaRDD() + jpyrdd = self._sc._jvm.PythonRDD.javaToPython(jPred) + return RDD(jpyrdd, self._sc, BatchedSerializer(ser, 1024)) + else: # Assume x is a single data point. - x_ = _serialize_double_vector(x) - return pythonAPI.predictDecisionTreeModel(self._java_model, x_) + bytes = bytearray(ser.dumps(_convert_to_vector(x))) + vec = self._sc._jvm.SerDe.loads(bytes) + return self._java_model.predict(vec) def numNodes(self): return self._java_model.numNodes() @@ -77,7 +76,7 @@ def numNodes(self): def depth(self): return self._java_model.depth() - def __str__(self): + def __repr__(self): return self._java_model.toString() @@ -90,52 +89,23 @@ class DecisionTree(object): EXPERIMENTAL: This is an experimental API. It will probably be modified for Spark v1.2. - Example usage: - - >>> from numpy import array - >>> import sys - >>> from pyspark.mllib.regression import LabeledPoint - >>> from pyspark.mllib.tree import DecisionTree - >>> from pyspark.mllib.linalg import SparseVector - >>> - >>> data = [ - ... LabeledPoint(0.0, [0.0]), - ... LabeledPoint(1.0, [1.0]), - ... LabeledPoint(1.0, [2.0]), - ... LabeledPoint(1.0, [3.0]) - ... ] - >>> categoricalFeaturesInfo = {} # no categorical features - >>> model = DecisionTree.trainClassifier(sc.parallelize(data), numClasses=2, - ... categoricalFeaturesInfo=categoricalFeaturesInfo) - >>> sys.stdout.write(model) - DecisionTreeModel classifier - If (feature 0 <= 0.5) - Predict: 0.0 - Else (feature 0 > 0.5) - Predict: 1.0 - >>> model.predict(array([1.0])) > 0 - True - >>> model.predict(array([0.0])) == 0 - True - >>> sparse_data = [ - ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), - ... LabeledPoint(1.0, SparseVector(2, {1: 1.0})), - ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), - ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) - ... ] - >>> - >>> model = DecisionTree.trainRegressor(sc.parallelize(sparse_data), - ... categoricalFeaturesInfo=categoricalFeaturesInfo) - >>> model.predict(array([0.0, 1.0])) == 1 - True - >>> model.predict(array([0.0, 0.0])) == 0 - True - >>> model.predict(SparseVector(2, {1: 1.0})) == 1 - True - >>> model.predict(SparseVector(2, {1: 0.0})) == 0 - True """ + @staticmethod + def _train(data, type, numClasses, categoricalFeaturesInfo, + impurity="gini", maxDepth=5, maxBins=32, minInstancesPerNode=1, + minInfoGain=0.0): + first = data.first() + assert isinstance(first, LabeledPoint), "the data should be RDD of LabeledPoint" + sc = data.context + jrdd = data._to_java_object_rdd() + cfiMap = MapConverter().convert(categoricalFeaturesInfo, + sc._gateway._gateway_client) + model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( + jrdd, type, numClasses, cfiMap, + impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) + return DecisionTreeModel(sc, model) + @staticmethod def trainClassifier(data, numClasses, categoricalFeaturesInfo, impurity="gini", maxDepth=5, maxBins=32, minInstancesPerNode=1, @@ -159,18 +129,34 @@ def trainClassifier(data, numClasses, categoricalFeaturesInfo, the parent split :param minInfoGain: Min info gain required to create a split :return: DecisionTreeModel + + Example usage: + + >>> from numpy import array + >>> from pyspark.mllib.regression import LabeledPoint + >>> from pyspark.mllib.tree import DecisionTree + >>> from pyspark.mllib.linalg import SparseVector + >>> + >>> data = [ + ... LabeledPoint(0.0, [0.0]), + ... LabeledPoint(1.0, [1.0]), + ... LabeledPoint(1.0, [2.0]), + ... LabeledPoint(1.0, [3.0]) + ... ] + >>> model = DecisionTree.trainClassifier(sc.parallelize(data), 2, {}) + >>> print model, # it already has newline + DecisionTreeModel classifier + If (feature 0 <= 0.5) + Predict: 0.0 + Else (feature 0 > 0.5) + Predict: 1.0 + >>> model.predict(array([1.0])) > 0 + True + >>> model.predict(array([0.0])) == 0 + True """ - sc = data.context - dataBytes = _get_unmangled_labeled_point_rdd(data) - categoricalFeaturesInfoJMap = \ - MapConverter().convert(categoricalFeaturesInfo, - sc._gateway._gateway_client) - model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( - dataBytes._jrdd, "classification", - numClasses, categoricalFeaturesInfoJMap, - impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) - dataBytes.unpersist() - return DecisionTreeModel(sc, model) + return DecisionTree._train(data, "classification", numClasses, categoricalFeaturesInfo, + impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) @staticmethod def trainRegressor(data, categoricalFeaturesInfo, @@ -194,18 +180,33 @@ def trainRegressor(data, categoricalFeaturesInfo, the parent split :param minInfoGain: Min info gain required to create a split :return: DecisionTreeModel + + Example usage: + + >>> from numpy import array + >>> from pyspark.mllib.regression import LabeledPoint + >>> from pyspark.mllib.tree import DecisionTree + >>> from pyspark.mllib.linalg import SparseVector + >>> + >>> sparse_data = [ + ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), + ... LabeledPoint(1.0, SparseVector(2, {1: 1.0})), + ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), + ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) + ... ] + >>> + >>> model = DecisionTree.trainRegressor(sc.parallelize(sparse_data), {}) + >>> model.predict(array([0.0, 1.0])) == 1 + True + >>> model.predict(array([0.0, 0.0])) == 0 + True + >>> model.predict(SparseVector(2, {1: 1.0})) == 1 + True + >>> model.predict(SparseVector(2, {1: 0.0})) == 0 + True """ - sc = data.context - dataBytes = _get_unmangled_labeled_point_rdd(data) - categoricalFeaturesInfoJMap = \ - MapConverter().convert(categoricalFeaturesInfo, - sc._gateway._gateway_client) - model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( - dataBytes._jrdd, "regression", - 0, categoricalFeaturesInfoJMap, - impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) - dataBytes.unpersist() - return DecisionTreeModel(sc, model) + return DecisionTree._train(data, "regression", 0, categoricalFeaturesInfo, + impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) def _test(): diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 1c7b8c809ab5b..8233d4e81f1ca 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -18,11 +18,10 @@ import numpy as np import warnings -from pyspark.mllib.linalg import Vectors, SparseVector -from pyspark.mllib.regression import LabeledPoint -from pyspark.mllib._common import _convert_vector, _deserialize_labeled_point from pyspark.rdd import RDD -from pyspark.serializers import NoOpSerializer +from pyspark.serializers import BatchedSerializer, PickleSerializer +from pyspark.mllib.linalg import Vectors, SparseVector, _convert_to_vector +from pyspark.mllib.regression import LabeledPoint class MLUtils(object): @@ -32,15 +31,12 @@ class MLUtils(object): """ @staticmethod - def _parse_libsvm_line(line, multiclass): - warnings.warn("deprecated", DeprecationWarning) - return _parse_libsvm_line(line) - - @staticmethod - def _parse_libsvm_line(line): + def _parse_libsvm_line(line, multiclass=None): """ Parses a line in LIBSVM format into (label, indices, values). """ + if multiclass is not None: + warnings.warn("deprecated", DeprecationWarning) items = line.split(None) label = float(items[0]) nnz = len(items) - 1 @@ -55,27 +51,20 @@ def _parse_libsvm_line(line): @staticmethod def _convert_labeled_point_to_libsvm(p): """Converts a LabeledPoint to a string in LIBSVM format.""" + assert isinstance(p, LabeledPoint) items = [str(p.label)] - v = _convert_vector(p.features) - if type(v) == np.ndarray: - for i in xrange(len(v)): - items.append(str(i + 1) + ":" + str(v[i])) - elif type(v) == SparseVector: + v = _convert_to_vector(p.features) + if isinstance(v, SparseVector): nnz = len(v.indices) for i in xrange(nnz): items.append(str(v.indices[i] + 1) + ":" + str(v.values[i])) else: - raise TypeError("_convert_labeled_point_to_libsvm needs either ndarray or SparseVector" - " but got " % type(v)) + for i in xrange(len(v)): + items.append(str(i + 1) + ":" + str(v[i])) return " ".join(items) @staticmethod - def loadLibSVMFile(sc, path, multiclass=False, numFeatures=-1, minPartitions=None): - warnings.warn("deprecated", DeprecationWarning) - return loadLibSVMFile(sc, path, numFeatures, minPartitions) - - @staticmethod - def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None): + def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None, multiclass=None): """ Loads labeled data in the LIBSVM format into an RDD of LabeledPoint. The LIBSVM format is a text-based format used by @@ -122,6 +111,8 @@ def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None): >>> print examples[2] (-1.0,(6,[1,3,5],[4.0,5.0,6.0])) """ + if multiclass is not None: + warnings.warn("deprecated", DeprecationWarning) lines = sc.textFile(path, minPartitions) parsed = lines.map(lambda l: MLUtils._parse_libsvm_line(l)) @@ -182,9 +173,9 @@ def loadLabeledPoints(sc, path, minPartitions=None): (0.0,[1.01,2.02,3.03]) """ minPartitions = minPartitions or min(sc.defaultParallelism, 2) - jSerialized = sc._jvm.PythonMLLibAPI().loadLabeledPoints(sc._jsc, path, minPartitions) - serialized = RDD(jSerialized, sc, NoOpSerializer()) - return serialized.map(lambda bytes: _deserialize_labeled_point(bytearray(bytes))) + jrdd = sc._jvm.PythonMLLibAPI().loadLabeledPoints(sc._jsc, path, minPartitions) + jpyrdd = sc._jvm.PythonRDD.javaToPython(jrdd) + return RDD(jpyrdd, sc, BatchedSerializer(PickleSerializer())) def _test(): diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index cb09c191bed71..8ef233bc80c5c 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -34,7 +34,7 @@ from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ - PickleSerializer, pack_long, CompressedSerializer + PickleSerializer, pack_long, AutoBatchedSerializer from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup from pyspark.statcounter import StatCounter @@ -1927,10 +1927,10 @@ def _to_java_object_rdd(self): It will convert each Python object into Java object by Pyrolite, whenever the RDD is serialized in batch or not. """ - if not self._is_pickled(): - self = self._reserialize(BatchedSerializer(PickleSerializer(), 1024)) - batched = isinstance(self._jrdd_deserializer, BatchedSerializer) - return self.ctx._jvm.PythonRDD.pythonToJava(self._jrdd, batched) + rdd = self._reserialize(AutoBatchedSerializer(PickleSerializer())) \ + if not self._is_pickled() else self + is_batch = isinstance(rdd._jrdd_deserializer, BatchedSerializer) + return self.ctx._jvm.PythonRDD.pythonToJava(rdd._jrdd, is_batch) def countApprox(self, timeout, confidence=0.95): """ @@ -2061,8 +2061,12 @@ def _jrdd(self): self._jrdd_deserializer = NoOpSerializer() command = (self.func, self._prev_jrdd_deserializer, self._jrdd_deserializer) + # the serialized command will be compressed by broadcast ser = CloudPickleSerializer() pickled_command = ser.dumps(command) + if pickled_command > (1 << 20): # 1M + broadcast = self.ctx.broadcast(pickled_command) + pickled_command = ser.dumps(broadcast) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], self.ctx._gateway._gateway_client) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 44ac5642836e0..2672da36c1f50 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -68,6 +68,7 @@ import types import collections import zlib +import itertools from pyspark import cloudpickle @@ -214,6 +215,41 @@ def __str__(self): return "BatchedSerializer<%s>" % str(self.serializer) +class AutoBatchedSerializer(BatchedSerializer): + """ + Choose the size of batch automatically based on the size of object + """ + + def __init__(self, serializer, bestSize=1 << 20): + BatchedSerializer.__init__(self, serializer, -1) + self.bestSize = bestSize + + def dump_stream(self, iterator, stream): + batch, best = 1, self.bestSize + iterator = iter(iterator) + while True: + vs = list(itertools.islice(iterator, batch)) + if not vs: + break + + bytes = self.serializer.dumps(vs) + write_int(len(bytes), stream) + stream.write(bytes) + + size = len(bytes) + if size < best: + batch *= 2 + elif size > best * 10 and batch > 1: + batch /= 2 + + def __eq__(self, other): + return (isinstance(other, AutoBatchedSerializer) and + other.serializer == self.serializer) + + def __str__(self): + return "BatchedSerializer<%s>" % str(self.serializer) + + class CartesianDeserializer(FramedSerializer): """ diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 8f6dbab240c7b..653195ea438cf 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -27,7 +27,7 @@ from array import array from operator import itemgetter -from pyspark.rdd import RDD, PipelinedRDD +from pyspark.rdd import RDD from pyspark.serializers import BatchedSerializer, PickleSerializer, CloudPickleSerializer from pyspark.storagelevel import StorageLevel from pyspark.traceback_utils import SCCallSiteSync @@ -440,6 +440,7 @@ def _parse_datatype_string(datatype_string): float: DoubleType, str: StringType, unicode: StringType, + bytearray: BinaryType, decimal.Decimal: DecimalType, datetime.datetime: TimestampType, datetime.date: TimestampType, @@ -690,11 +691,12 @@ def _infer_schema_type(obj, dataType): ByteType: (int, long), ShortType: (int, long), IntegerType: (int, long), - LongType: (long,), + LongType: (int, long), FloatType: (float,), DoubleType: (float,), DecimalType: (decimal.Decimal,), StringType: (str, unicode), + BinaryType: (bytearray,), TimestampType: (datetime.datetime,), ArrayType: (list, tuple, array), MapType: (dict,), @@ -728,9 +730,9 @@ def _verify_type(obj, dataType): return _type = type(dataType) - if _type not in _acceptable_types: - return + assert _type in _acceptable_types, "unkown datatype: %s" % dataType + # subclass of them can not be deserialized in JVM if type(obj) not in _acceptable_types[_type]: raise TypeError("%s can not accept abject in type %s" % (dataType, type(obj))) @@ -975,7 +977,11 @@ def registerFunction(self, name, f, returnType=StringType()): command = (func, BatchedSerializer(PickleSerializer(), 1024), BatchedSerializer(PickleSerializer(), 1024)) - pickled_command = CloudPickleSerializer().dumps(command) + ser = CloudPickleSerializer() + pickled_command = ser.dumps(command) + if pickled_command > (1 << 20): # 1M + broadcast = self._sc.broadcast(pickled_command) + pickled_command = ser.dumps(broadcast) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self._sc._pickled_broadcast_vars], self._sc._gateway._gateway_client) @@ -1117,6 +1123,11 @@ def applySchema(self, rdd, schema): # take the first few rows to verify schema rows = rdd.take(10) + # Row() cannot been deserialized by Pyrolite + if rows and isinstance(rows[0], tuple) and rows[0].__class__.__name__ == 'Row': + rdd = rdd.map(tuple) + rows = rdd.take(10) + for row in rows: _verify_type(row, schema) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 0b3854347ad2e..1b8afb763b26a 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -45,7 +45,7 @@ from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ CloudPickleSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter -from pyspark.sql import SQLContext, IntegerType +from pyspark.sql import SQLContext, IntegerType, Row from pyspark import shuffle _have_scipy = False @@ -434,6 +434,12 @@ def test_large_broadcast(self): m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum() self.assertEquals(N, m) + def test_large_closure(self): + N = 1000000 + data = [float(i) for i in xrange(N)] + m = self.sc.parallelize(range(1), 1).map(lambda x: len(data)).sum() + self.assertEquals(N, m) + def test_zip_with_different_serializers(self): a = self.sc.parallelize(range(5)) b = self.sc.parallelize(range(100, 105)) @@ -653,6 +659,15 @@ def test_distinct(self): self.assertEquals(result.getNumPartitions(), 5) self.assertEquals(result.count(), 3) + def test_apply_schema_to_row(self): + srdd = self.sqlCtx.jsonRDD(self.sc.parallelize(["""{"a":2}"""])) + srdd2 = self.sqlCtx.applySchema(srdd.map(lambda x: x), srdd.schema()) + self.assertEqual(srdd.collect(), srdd2.collect()) + + rdd = self.sc.parallelize(range(10)).map(lambda x: Row(a=x)) + srdd3 = self.sqlCtx.applySchema(rdd, srdd.schema()) + self.assertEqual(10, srdd3.count()) + class TestIO(PySparkTestCase): @@ -1092,7 +1107,7 @@ def test_reserialization(self): def test_unbatched_save_and_read(self): basepath = self.tempdir.name ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')] - self.sc.parallelize(ei, numSlices=len(ei)).saveAsSequenceFile( + self.sc.parallelize(ei, len(ei)).saveAsSequenceFile( basepath + "/unbatched/") unbatched_sequence = sorted(self.sc.sequenceFile( @@ -1138,7 +1153,7 @@ def test_malformed_RDD(self): basepath = self.tempdir.name # non-batch-serialized RDD[[(K, V)]] should be rejected data = [[(1, "a")], [(2, "aa")], [(3, "aaa")]] - rdd = self.sc.parallelize(data, numSlices=len(data)) + rdd = self.sc.parallelize(data, len(data)) self.assertRaises(Exception, lambda: rdd.saveAsSequenceFile( basepath + "/malformed/sequence")) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 252176ac65fec..d6c06e2dbef62 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -77,10 +77,12 @@ def main(infile, outfile): _broadcastRegistry[bid] = Broadcast(bid, value) else: bid = - bid - 1 - _broadcastRegistry.remove(bid) + _broadcastRegistry.pop(bid) _accumulatorRegistry.clear() command = pickleSer._read_with_length(infile) + if isinstance(command, Broadcast): + command = pickleSer.loads(command.value) (func, deserializer, serializer) = command init_time = time.time() iterator = deserializer.load_stream(infile) diff --git a/python/run-tests b/python/run-tests index a67e5a99fbdcc..a7ec270c7da21 100755 --- a/python/run-tests +++ b/python/run-tests @@ -73,7 +73,6 @@ run_test "pyspark/serializers.py" unset PYSPARK_DOC_TEST run_test "pyspark/shuffle.py" run_test "pyspark/tests.py" -run_test "pyspark/mllib/_common.py" run_test "pyspark/mllib/classification.py" run_test "pyspark/mllib/clustering.py" run_test "pyspark/mllib/linalg.py" diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index 4ce40fe750384..ba953e763faab 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -27,7 +27,7 @@ set -o posix FWDIR="$(cd "`dirname "$0"`"/..; pwd)" CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" -CLASS_NOT_FOUND_EXIT_STATUS=1 +CLASS_NOT_FOUND_EXIT_STATUS=101 function usage { echo "Usage: ./sbin/start-thriftserver [options] [thrift server options]" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index ca69531c69a77..862f78702c4e6 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -151,7 +151,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} | UNION ~ opt(DISTINCT) ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } ) - | insert | cache + | insert | cache | unCache ) protected lazy val select: Parser[LogicalPlan] = @@ -183,9 +183,17 @@ class SqlParser extends StandardTokenParsers with PackratParsers { } protected lazy val cache: Parser[LogicalPlan] = - (CACHE ^^^ true | UNCACHE ^^^ false) ~ TABLE ~ ident ^^ { - case doCache ~ _ ~ tableName => CacheCommand(tableName, doCache) + CACHE ~ TABLE ~> ident ~ opt(AS ~> select) <~ opt(";") ^^ { + case tableName ~ None => + CacheCommand(tableName, true) + case tableName ~ Some(plan) => + CacheTableAsSelectCommand(tableName, plan) } + + protected lazy val unCache: Parser[LogicalPlan] = + UNCACHE ~ TABLE ~> ident <~ opt(";") ^^ { + case tableName => CacheCommand(tableName, false) + } protected lazy val projections: Parser[Seq[Expression]] = repsep(projection, ",") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 574d96d92942b..71810b798bd04 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -37,6 +37,8 @@ object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, true class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Boolean) extends RuleExecutor[LogicalPlan] with HiveTypeCoercion { + val resolver = if (caseSensitive) caseSensitiveResolution else caseInsensitiveResolution + // TODO: pass this in as a parameter. val fixedPoint = FixedPoint(100) @@ -48,8 +50,6 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool lazy val batches: Seq[Batch] = Seq( Batch("MultiInstanceRelations", Once, NewRelationInstances), - Batch("CaseInsensitiveAttributeReferences", Once, - (if (caseSensitive) Nil else LowercaseAttributeReferences :: Nil) : _*), Batch("Resolution", fixedPoint, ResolveReferences :: ResolveRelations :: @@ -98,23 +98,6 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } } - /** - * Makes attribute naming case insensitive by turning all UnresolvedAttributes to lowercase. - */ - object LowercaseAttributeReferences extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case UnresolvedRelation(databaseName, name, alias) => - UnresolvedRelation(databaseName, name, alias.map(_.toLowerCase)) - case Subquery(alias, child) => Subquery(alias.toLowerCase, child) - case q: LogicalPlan => q transformExpressions { - case s: Star => s.copy(table = s.table.map(_.toLowerCase)) - case UnresolvedAttribute(name) => UnresolvedAttribute(name.toLowerCase) - case Alias(c, name) => Alias(c, name.toLowerCase)() - case GetField(c, name) => GetField(c, name.toLowerCase) - } - } - } - /** * Replaces [[UnresolvedAttribute]]s with concrete * [[catalyst.expressions.AttributeReference AttributeReferences]] from a logical plan node's @@ -127,7 +110,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool q transformExpressions { case u @ UnresolvedAttribute(name) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. - val result = q.resolveChildren(name).getOrElse(u) + val result = q.resolveChildren(name, resolver).getOrElse(u) logDebug(s"Resolving $u to $result") result } @@ -144,7 +127,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case s @ Sort(ordering, p @ Project(projectList, child)) if !s.resolved && p.resolved => val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) - val resolved = unresolved.flatMap(child.resolveChildren) + val resolved = unresolved.flatMap(child.resolve(_, resolver)) val requiredAttributes = AttributeSet(resolved.collect { case a: Attribute => a }) val missingInProject = requiredAttributes -- p.output @@ -154,6 +137,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool Sort(ordering, Project(projectList ++ missingInProject, child))) } else { + logDebug(s"Failed to find $missingInProject in ${p.output.mkString(", ")}") s // Nothing we can do here. Return original plan. } case s @ Sort(ordering, a @ Aggregate(grouping, aggs, child)) if !s.resolved && a.resolved => @@ -165,7 +149,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool ) logDebug(s"Grouping expressions: $groupingRelation") - val resolved = unresolved.flatMap(groupingRelation.resolve) + val resolved = unresolved.flatMap(groupingRelation.resolve(_, resolver)) val missingInAggs = resolved.filterNot(a.outputSet.contains) logDebug(s"Resolved: $resolved Missing in aggs: $missingInAggs") if (missingInAggs.nonEmpty) { @@ -258,14 +242,14 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case p @ Project(projectList, child) if containsStar(projectList) => Project( projectList.flatMap { - case s: Star => s.expand(child.output) + case s: Star => s.expand(child.output, resolver) case o => o :: Nil }, child) case t: ScriptTransformation if containsStar(t.input) => t.copy( input = t.input.flatMap { - case s: Star => s.expand(t.child.output) + case s: Star => s.expand(t.child.output, resolver) case o => o :: Nil } ) @@ -273,7 +257,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case a: Aggregate if containsStar(a.aggregateExpressions) => a.copy( aggregateExpressions = a.aggregateExpressions.flatMap { - case s: Star => s.expand(a.child.output) + case s: Star => s.expand(a.child.output, resolver) case o => o :: Nil } ) @@ -290,13 +274,11 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool /** * Removes [[catalyst.plans.logical.Subquery Subquery]] operators from the plan. Subqueries are * only required to provide scoping information for attributes and can be removed once analysis is - * complete. Similarly, this node also removes - * [[catalyst.plans.logical.LowerCaseSchema LowerCaseSchema]] operators. + * complete. */ object EliminateAnalysisOperators extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case Subquery(_, child) => child - case LowerCaseSchema(child) => child } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala index 9f37ca904ffeb..3f672a3e0fd91 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -22,4 +22,14 @@ package org.apache.spark.sql.catalyst * Analysis consists of translating [[UnresolvedAttribute]]s and [[UnresolvedRelation]]s * into fully typed objects using information in a schema [[Catalog]]. */ -package object analysis +package object analysis { + + /** + * Responsible for resolving which identifiers refer to the same entity. For example, by using + * case insensitive equality. + */ + type Resolver = (String, String) => Boolean + + val caseInsensitiveResolution = (a: String, b: String) => a.equalsIgnoreCase(b) + val caseSensitiveResolution = (a: String, b: String) => a == b +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index a2c61c65487cb..67570a6f73c36 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -54,6 +54,7 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo override def newInstance = this override def withNullability(newNullability: Boolean) = this override def withQualifiers(newQualifiers: Seq[String]) = this + override def withName(newName: String) = UnresolvedAttribute(name) // Unresolved attributes are transient at compile time and don't get evaluated during execution. override def eval(input: Row = null): EvaluatedType = @@ -97,13 +98,14 @@ case class Star( override def newInstance = this override def withNullability(newNullability: Boolean) = this override def withQualifiers(newQualifiers: Seq[String]) = this + override def withName(newName: String) = this - def expand(input: Seq[Attribute]): Seq[NamedExpression] = { + def expand(input: Seq[Attribute], resolver: Resolver): Seq[NamedExpression] = { val expandedAttributes: Seq[Attribute] = table match { // If there is no table specified, use all input attributes. case None => input // If there is a table, pick out attributes that are part of this table. - case Some(t) => input.filter(_.qualifiers contains t) + case Some(t) => input.filter(_.qualifiers.filter(resolver(_, t)).nonEmpty) } val mappedAttributes = expandedAttributes.map(mapFunction).zip(input).map { case (n: NamedExpression, _) => n diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 0ad2b30cf9c1f..0379275121bf2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -245,6 +245,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { } private[this] lazy val cast: Any => Any = dataType match { + case dt if dt == child.dataType => identity[Any] case StringType => castToString case BinaryType => castToBinary case DecimalType => castToDecimal diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 7c4b9d4847e26..59fb0311a9c44 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -59,6 +59,7 @@ abstract class Attribute extends NamedExpression { def withNullability(newNullability: Boolean): Attribute def withQualifiers(newQualifiers: Seq[String]): Attribute + def withName(newName: String): Attribute def toAttribute = this def newInstance: Attribute @@ -86,7 +87,6 @@ case class Alias(child: Expression, name: String) override def dataType = child.dataType override def nullable = child.nullable - override def toAttribute = { if (resolved) { AttributeReference(name, child.dataType, child.nullable)(exprId, qualifiers) @@ -144,6 +144,14 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea } } + override def withName(newName: String): AttributeReference = { + if (name == newName) { + this + } else { + AttributeReference(newName, dataType, nullable)(exprId, qualifiers) + } + } + /** * Returns a copy of this [[AttributeReference]] with new qualifiers. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index ede431ad4ab27..28d863e58beca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -17,13 +17,15 @@ package org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.types.StructType import org.apache.spark.sql.catalyst.trees -abstract class LogicalPlan extends QueryPlan[LogicalPlan] { +abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { self: Product => /** @@ -75,20 +77,25 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { * nodes of this LogicalPlan. The attribute is expressed as * as string in the following form: `[scope].AttributeName.[nested].[fields]...`. */ - def resolveChildren(name: String): Option[NamedExpression] = - resolve(name, children.flatMap(_.output)) + def resolveChildren(name: String, resolver: Resolver): Option[NamedExpression] = + resolve(name, children.flatMap(_.output), resolver) /** * Optionally resolves the given string to a [[NamedExpression]] based on the output of this * LogicalPlan. The attribute is expressed as string in the following form: * `[scope].AttributeName.[nested].[fields]...`. */ - def resolve(name: String): Option[NamedExpression] = - resolve(name, output) + def resolve(name: String, resolver: Resolver): Option[NamedExpression] = + resolve(name, output, resolver) /** Performs attribute resolution given a name and a sequence of possible attributes. */ - protected def resolve(name: String, input: Seq[Attribute]): Option[NamedExpression] = { + protected def resolve( + name: String, + input: Seq[Attribute], + resolver: Resolver): Option[NamedExpression] = { + val parts = name.split("\\.") + // Collect all attributes that are output by this nodes children where either the first part // matches the name or where the first part matches the scope and the second part matches the // name. Return these matches along with any remaining parts, which represent dotted access to @@ -96,21 +103,69 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { val options = input.flatMap { option => // If the first part of the desired name matches a qualifier for this possible match, drop it. val remainingParts = - if (option.qualifiers.contains(parts.head) && parts.size > 1) parts.drop(1) else parts - if (option.name == remainingParts.head) (option, remainingParts.tail.toList) :: Nil else Nil + if (option.qualifiers.find(resolver(_, parts.head)).nonEmpty && parts.size > 1) { + parts.drop(1) + } else { + parts + } + + if (resolver(option.name, remainingParts.head)) { + // Preserve the case of the user's attribute reference. + (option.withName(remainingParts.head), remainingParts.tail.toList) :: Nil + } else { + Nil + } } options.distinct match { - case Seq((a, Nil)) => Some(a) // One match, no nested fields, use it. + // One match, no nested fields, use it. + case Seq((a, Nil)) => Some(a) + // One match, but we also need to extract the requested nested field. case Seq((a, nestedFields)) => - Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) - case Seq() => None // No matches. + val aliased = + Alias( + resolveNesting(nestedFields, a, resolver), + nestedFields.last)() // Preserve the case of the user's field access. + Some(aliased) + + // No matches. + case Seq() => + logTrace(s"Could not find $name in ${input.mkString(", ")}") + None + + // More than one match. case ambiguousReferences => throw new TreeNodeException( this, s"Ambiguous references to $name: ${ambiguousReferences.mkString(",")}") } } + + /** + * Given a list of successive nested field accesses, and a based expression, attempt to resolve + * the actual field lookups on this expression. + */ + private def resolveNesting( + nestedFields: List[String], + expression: Expression, + resolver: Resolver): Expression = { + + (nestedFields, expression.dataType) match { + case (Nil, _) => expression + case (requestedField :: rest, StructType(fields)) => + val actualField = fields.filter(f => resolver(f.name, requestedField)) + actualField match { + case Seq() => + sys.error( + s"No such struct field $requestedField in ${fields.map(_.name).mkString(", ")}") + case Seq(singleMatch) => + resolveNesting(rest, GetField(expression, singleMatch.name), resolver) + case multipleMatches => + sys.error(s"Ambiguous reference to fields ${multipleMatches.mkString(", ")}") + } + case (_, dt) => sys.error(s"Can't access nested field in type $dt") + } + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 5d10754c7b028..391508279bb80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -148,38 +148,23 @@ case class Aggregate( case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { override def output = child.output + + override lazy val statistics: Statistics = + if (output.forall(_.dataType.isInstanceOf[NativeType])) { + val limit = limitExpr.eval(null).asInstanceOf[Int] + val sizeInBytes = (limit: Long) * output.map { a => + NativeType.defaultSizeOf(a.dataType.asInstanceOf[NativeType]) + }.sum + Statistics(sizeInBytes = sizeInBytes) + } else { + Statistics(sizeInBytes = children.map(_.statistics).map(_.sizeInBytes).product) + } } case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { override def output = child.output.map(_.withQualifiers(alias :: Nil)) } -/** - * Converts the schema of `child` to all lowercase, together with LowercaseAttributeReferences - * this allows for optional case insensitive attribute resolution. This node can be elided after - * analysis. - */ -case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode { - protected def lowerCaseSchema(dataType: DataType): DataType = dataType match { - case StructType(fields) => - StructType(fields.map(f => - StructField(f.name.toLowerCase(), lowerCaseSchema(f.dataType), f.nullable))) - case ArrayType(elemType, containsNull) => ArrayType(lowerCaseSchema(elemType), containsNull) - case otherType => otherType - } - - override val output = child.output.map { - case a: AttributeReference => - AttributeReference( - a.name.toLowerCase, - lowerCaseSchema(a.dataType), - a.nullable)( - a.exprId, - a.qualifiers) - case other => other - } -} - case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: LogicalPlan) extends UnaryNode { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index a01809c1fc5e2..8366639fa0e8b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -75,3 +75,8 @@ case class DescribeCommand( AttributeReference("data_type", StringType, nullable = false)(), AttributeReference("comment", StringType, nullable = false)()) } + +/** + * Returned for the "CACHE TABLE tableName AS SELECT .." command. + */ +case class CacheTableAsSelectCommand(tableName: String, plan: LogicalPlan) extends Command diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index 49520b7678e90..e3050e5397937 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -122,6 +122,16 @@ object NativeType { IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) def unapply(dt: DataType): Boolean = all.contains(dt) + + val defaultSizeOf: Map[NativeType, Int] = Map( + IntegerType -> 4, + BooleanType -> 1, + LongType -> 8, + DoubleType -> 8, + FloatType -> 4, + ShortType -> 2, + ByteType -> 1, + StringType -> 4096) } trait PrimitiveType extends DataType { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 7dbaf7faff0c0..b245e1a863cc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -246,7 +246,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def registerRDDAsTable(rdd: SchemaRDD, tableName: String): Unit = { - catalog.registerTable(None, tableName, rdd.queryExecution.analyzed) + catalog.registerTable(None, tableName, rdd.queryExecution.logical) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 3bc5dce095511..3b873f7c62cb6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -45,9 +45,8 @@ import org.apache.spark.api.java.JavaRDD * explicitly using the `createSchemaRDD` function on a [[SQLContext]]. * * A `SchemaRDD` can also be created by loading data in from external sources. - * Examples are loading data from Parquet files by using by using the - * `parquetFile` method on [[SQLContext]], and loading JSON datasets - * by using `jsonFile` and `jsonRDD` methods on [[SQLContext]]. + * Examples are loading data from Parquet files by using the `parquetFile` method on [[SQLContext]] + * and loading JSON datasets by using `jsonFile` and `jsonRDD` methods on [[SQLContext]]. * * == SQL Queries == * A SchemaRDD can be registered as a table in the [[SQLContext]] that was used to create it. Once diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 7943d6e1b6fb5..45687d960404c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -305,6 +305,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { Seq(execution.ExplainCommand(logicalPlan, plan.output, extended)(context)) case logical.CacheCommand(tableName, cache) => Seq(execution.CacheCommand(tableName, cache)(context)) + case logical.CacheTableAsSelectCommand(tableName, plan) => + Seq(execution.CacheTableAsSelectCommand(tableName, plan)) case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 94543fc95b470..c2f48a902a3e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -166,3 +166,21 @@ case class DescribeCommand(child: SparkPlan, output: Seq[Attribute])( child.output.map(field => Row(field.name, field.dataType.toString, null)) } } + +/** + * :: DeveloperApi :: + */ +@DeveloperApi +case class CacheTableAsSelectCommand(tableName: String, logicalPlan: LogicalPlan) + extends LeafNode with Command { + + override protected[sql] lazy val sideEffectResult = { + import sqlContext._ + logicalPlan.registerTempTable(tableName) + cacheTable(tableName) + Seq.empty[Row] + } + + override def output: Seq[Attribute] = Seq.empty + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index befef46d93973..591592841e9fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -119,4 +119,17 @@ class CachedTableSuite extends QueryTest { } assert(!TestSQLContext.isCached("testData"), "Table 'testData' should not be cached") } + + test("CACHE TABLE tableName AS SELECT Star Table") { + TestSQLContext.sql("CACHE TABLE testCacheTable AS SELECT * FROM testData") + TestSQLContext.sql("SELECT * FROM testCacheTable WHERE key = 1").collect() + assert(TestSQLContext.isCached("testCacheTable"), "Table 'testCacheTable' should be cached") + TestSQLContext.uncacheTable("testCacheTable") + } + + test("'CACHE TABLE tableName AS SELECT ..'") { + TestSQLContext.sql("CACHE TABLE testCacheTable AS SELECT * FROM testData") + assert(TestSQLContext.isCached("testCacheTable"), "Table 'testCacheTable' should be cached") + TestSQLContext.uncacheTable("testCacheTable") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 67563b6c55f4b..08376eb5e5c4e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.{ShuffledHashJoin, BroadcastHashJoin} import org.apache.spark.sql.test._ import org.scalatest.BeforeAndAfterAll import java.util.TimeZone @@ -380,7 +381,6 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SPARK-3349 partitioning after limit") { - /* sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC") .limit(2) .registerTempTable("subset1") @@ -395,7 +395,6 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { sql("SELECT * FROM lowerCaseData INNER JOIN subset2 ON subset2.n = lowerCaseData.n"), (1, "a", 1) :: (2, "b", 2) :: Nil) - */ } test("mixed-case keywords") { @@ -649,24 +648,24 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { (3, null) :: (4, 2147483644) :: Nil) } - + test("SPARK-3423 BETWEEN") { checkAnswer( sql("SELECT key, value FROM testData WHERE key BETWEEN 5 and 7"), Seq((5, "5"), (6, "6"), (7, "7")) ) - + checkAnswer( sql("SELECT key, value FROM testData WHERE key BETWEEN 7 and 7"), Seq((7, "7")) ) - + checkAnswer( sql("SELECT key, value FROM testData WHERE key BETWEEN 9 and 7"), Seq() ) } - + test("cast boolean to string") { // TODO Ensure true/false string letter casing is consistent with Hive in all cases. checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 37d64f0de7bab..bfbf431a11913 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -22,7 +22,7 @@ import org.scalatest.FunSuite import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.execution +import org.apache.spark.sql.{SQLConf, execution} import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.planner._ @@ -57,4 +57,22 @@ class PlannerSuite extends FunSuite { val planned = HashAggregation(query) assert(planned.nonEmpty) } + + test("sizeInBytes estimation of limit operator for broadcast hash join optimization") { + val origThreshold = autoBroadcastJoinThreshold + setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, 81920.toString) + + // Using a threshold that is definitely larger than the small testing table (b) below + val a = testData.as('a) + val b = testData.limit(3).as('b) + val planned = a.join(b, Inner, Some("a.key".attr === "b.key".attr)).queryExecution.executedPlan + + val broadcastHashJoins = planned.collect { case join: BroadcastHashJoin => join } + val shuffledHashJoins = planned.collect { case join: ShuffledHashJoin => join } + + assert(broadcastHashJoins.size === 1, "Should use broadcast hash join") + assert(shuffledHashJoins.isEmpty, "Should not use shuffled hash join") + + setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, origThreshold.toString) + } } diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index ab487d673e813..556c984ad392b 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -801,6 +801,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_or", "udf_parse_url", "udf_PI", + "udf_pmod", "udf_positive", "udf_pow", "udf_power", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index e0be09e6793ea..3e1a7b71528e0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -244,15 +244,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /* A catalyst metadata catalog that points to the Hive Metastore. */ @transient - override protected[sql] lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog { - override def lookupRelation( - databaseName: Option[String], - tableName: String, - alias: Option[String] = None): LogicalPlan = { - - LowerCaseSchema(super.lookupRelation(databaseName, tableName, alias)) - } - } + override protected[sql] lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog // Note that HiveUDFs will be overridden by functions registered in this context. @transient diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 2c0db9be57e54..6b4399e852c7b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -129,14 +129,12 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - case p @ InsertIntoTable( - LowerCaseSchema(table: MetastoreRelation), _, child, _) => + case p @ InsertIntoTable(table: MetastoreRelation, _, child, _) => castChildOutput(p, table, child) case p @ logical.InsertIntoTable( - LowerCaseSchema( InMemoryRelation(_, _, _, - HiveTableScan(_, table, _))), _, child, _) => + HiveTableScan(_, table, _)), _, child, _) => castChildOutput(p, table, child) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 21ecf17028dbc..0aa6292c0184e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -229,7 +229,12 @@ private[hive] object HiveQl { SetCommand(Some(key), Some(value)) } } else if (sql.trim.toLowerCase.startsWith("cache table")) { - CacheCommand(sql.trim.drop(12).trim, true) + sql.trim.drop(12).trim.split(" ").toSeq match { + case Seq(tableName) => + CacheCommand(tableName, true) + case Seq(tableName, _, select @ _*) => + CacheTableAsSelectCommand(tableName, createPlan(select.mkString(" ").trim)) + } } else if (sql.trim.toLowerCase.startsWith("uncache table")) { CacheCommand(sql.trim.drop(14).trim, false) } else if (sql.trim.toLowerCase.startsWith("add jar")) { @@ -243,15 +248,7 @@ private[hive] object HiveQl { } else if (sql.trim.startsWith("!")) { ShellCommand(sql.drop(1)) } else { - val tree = getAst(sql) - if (nativeCommands contains tree.getText) { - NativeCommand(sql) - } else { - nodeToPlan(tree) match { - case NativePlaceholder => NativeCommand(sql) - case other => other - } - } + createPlan(sql) } } catch { case e: Exception => throw new ParseException(sql, e) @@ -262,6 +259,19 @@ private[hive] object HiveQl { """.stripMargin) } } + + /** Creates LogicalPlan for a given HiveQL string. */ + def createPlan(sql: String) = { + val tree = getAst(sql) + if (nativeCommands contains tree.getText) { + NativeCommand(sql) + } else { + nodeToPlan(tree) match { + case NativePlaceholder => NativeCommand(sql) + case other => other + } + } + } def parseDdl(ddl: String): Seq[Attribute] = { val tree = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 43dd3d234f73a..8ac17f37201a8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LowerCaseSchema} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.types.StringType import org.apache.spark.sql.columnar.InMemoryRelation import org.apache.spark.sql.execution.{DescribeCommand, OutputFaker, SparkPlan} @@ -55,7 +55,7 @@ private[hive] trait HiveStrategies { object ParquetConversion extends Strategy { implicit class LogicalPlanHacks(s: SchemaRDD) { def lowerCase = - new SchemaRDD(s.sqlContext, LowerCaseSchema(s.logicalPlan)) + new SchemaRDD(s.sqlContext, s.logicalPlan) def addPartitioningAttributes(attrs: Seq[Attribute]) = new SchemaRDD( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index 71ea774d77795..1017fe6d5396d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LowerCaseSchema import org.apache.spark.sql.execution.{SparkPlan, Command, LeafNode} import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.MetastoreRelation @@ -52,8 +51,7 @@ case class CreateTableAsSelect( sc.catalog.createTable(database, tableName, query.output, false) // Get the Metastore Relation sc.catalog.lookupRelation(Some(database), tableName, None) match { - case LowerCaseSchema(r: MetastoreRelation) => r - case o: MetastoreRelation => o + case r: MetastoreRelation => r } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 7d1ad53d8bdb3..19ff3b66ad7ed 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper + import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.common.`type`.HiveDecimal @@ -42,21 +44,23 @@ private[hive] abstract class HiveFunctionRegistry def lookupFunction(name: String, children: Seq[Expression]): Expression = { // We only look it up to see if it exists, but do not include it in the HiveUDF since it is // not always serializable. - val functionInfo: FunctionInfo = Option(FunctionRegistry.getFunctionInfo(name)).getOrElse( - sys.error(s"Couldn't find function $name")) + val functionInfo: FunctionInfo = + Option(FunctionRegistry.getFunctionInfo(name.toLowerCase)).getOrElse( + sys.error(s"Couldn't find function $name")) - val functionClassName = functionInfo.getFunctionClass.getName() + val functionClassName = functionInfo.getFunctionClass.getName if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) { val function = functionInfo.getFunctionClass.newInstance().asInstanceOf[UDF] val method = function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) - lazy val expectedDataTypes = method.getParameterTypes.map(javaClassToDataType) + val expectedDataTypes = method.getParameterTypes.map(javaClassToDataType) HiveSimpleUdf( functionClassName, children.zip(expectedDataTypes).map { case (e, NullType) => e + case (e, t) if (e.dataType == t) => e case (e, t) => Cast(e, t) } ) @@ -104,52 +108,27 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) @transient - lazy val dataType = javaClassToDataType(method.getReturnType) + protected lazy val arguments = children.map(c => toInspector(c.dataType)).toArray - protected lazy val wrappers: Array[(Any) => AnyRef] = method.getParameterTypes.map { argClass => - val primitiveClasses = Seq( - Integer.TYPE, classOf[java.lang.Integer], classOf[java.lang.String], java.lang.Double.TYPE, - classOf[java.lang.Double], java.lang.Long.TYPE, classOf[java.lang.Long], - classOf[HiveDecimal], java.lang.Byte.TYPE, classOf[java.lang.Byte], - classOf[java.sql.Timestamp] - ) - val matchingConstructor = argClass.getConstructors.find { c => - c.getParameterTypes.size == 1 && primitiveClasses.contains(c.getParameterTypes.head) - } + // Create parameter converters + @transient + protected lazy val conversionHelper = new ConversionHelper(method, arguments) - matchingConstructor match { - case Some(constructor) => - (a: Any) => { - logDebug( - s"Wrapping $a of type ${if (a == null) "null" else a.getClass.getName} $constructor.") - // We must make sure that primitives get boxed java style. - if (a == null) { - null - } else { - constructor.newInstance(a match { - case i: Int => i: java.lang.Integer - case bd: BigDecimal => new HiveDecimal(bd.underlying()) - case other: AnyRef => other - }).asInstanceOf[AnyRef] - } - } - case None => - (a: Any) => a match { - case wrapper => wrap(wrapper) - } - } + @transient + lazy val dataType = javaClassToDataType(method.getReturnType) + + def catalystToHive(value: Any): Object = value match { + // TODO need more types here? or can we use wrap() + case bd: BigDecimal => new HiveDecimal(bd.underlying()) + case d => d.asInstanceOf[Object] } // TODO: Finish input output types. override def eval(input: Row): Any = { - val evaluatedChildren = children.map(_.eval(input)) - // Wrap the function arguments in the expected types. - val args = evaluatedChildren.zip(wrappers).map { - case (arg, wrapper) => wrapper(arg) - } + val evaluatedChildren = children.map(c => catalystToHive(c.eval(input))) - // Invoke the udf and unwrap the result. - unwrap(method.invoke(function, args: _*)) + unwrap(FunctionRegistry.invoke(method, function, conversionHelper + .convertIfNecessary(evaluatedChildren: _*): _*)) } } diff --git a/sql/hive/src/test/resources/golden/Cast Timestamp to Timestamp in UDF-0-66952a3949d7544716fd1a675498b1fa b/sql/hive/src/test/resources/golden/Cast Timestamp to Timestamp in UDF-0-66952a3949d7544716fd1a675498b1fa new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/Cast Timestamp to Timestamp in UDF-0-66952a3949d7544716fd1a675498b1fa @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/database.table table.attr case insensitive-0-98b2e34c9134208e9fe7c62d33010005 b/sql/hive/src/test/resources/golden/database.table table.attr case insensitive-0-98b2e34c9134208e9fe7c62d33010005 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/database.table table.attr case insensitive-0-98b2e34c9134208e9fe7c62d33010005 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 8c8a8b124ac69..56bcd95eab4bc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -142,16 +142,25 @@ class HiveQuerySuite extends HiveComparisonTest { setConf("spark.sql.dialect", "sql") assert(sql("SELECT 1").collect() === Array(Seq(1))) setConf("spark.sql.dialect", "hiveql") - } test("Query expressed in HiveQL") { sql("FROM src SELECT key").collect() } + test("Query with constant folding the CAST") { + sql("SELECT CAST(CAST('123' AS binary) AS binary) FROM src LIMIT 1").collect() + } + createQueryTest("Constant Folding Optimization for AVG_SUM_COUNT", "SELECT AVG(0), SUM(0), COUNT(null), COUNT(value) FROM src GROUP BY key") + createQueryTest("Cast Timestamp to Timestamp in UDF", + """ + | SELECT DATEDIFF(CAST(value AS timestamp), CAST('2002-03-21 00:00:00' AS timestamp)) + | FROM src LIMIT 1 + """.stripMargin) + createQueryTest("Simple Average", "SELECT AVG(key) FROM src") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index b6be6bc1bfefe..ee9d08ff75450 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -36,6 +36,9 @@ class HiveResolutionSuite extends HiveComparisonTest { createQueryTest("database.table table.attr", "SELECT src.key FROM default.src ORDER BY key LIMIT 1") + createQueryTest("database.table table.attr case insensitive", + "SELECT SRC.Key FROM Default.Src ORDER BY key LIMIT 1") + createQueryTest("alias.attr", "SELECT a.key FROM src a ORDER BY key LIMIT 1") @@ -56,14 +59,18 @@ class HiveResolutionSuite extends HiveComparisonTest { TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) .registerTempTable("caseSensitivityTest") - sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") - - println(sql("SELECT * FROM casesensitivitytest one JOIN casesensitivitytest two ON one.a = two.a").queryExecution) - - sql("SELECT * FROM casesensitivitytest one JOIN casesensitivitytest two ON one.a = two.a").collect() + val query = sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") + assert(query.schema.fields.map(_.name) === Seq("a", "b", "A", "B", "a", "b", "A", "B"), + "The output schema did not preserve the case of the query.") + query.collect() + } - // TODO: sql("SELECT * FROM casesensitivitytest a JOIN casesensitivitytest b ON a.a = b.a") + ignore("case insensitivity with scala reflection joins") { + // Test resolution with Scala Reflection + TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) + .registerTempTable("caseSensitivityTest") + sql("SELECT * FROM casesensitivitytest a JOIN casesensitivitytest b ON a.a = b.a").collect() } test("nested repeated resolution") {