diff --git a/README.md b/README.md index af02339578195..c3afc4db9c63c 100644 --- a/README.md +++ b/README.md @@ -85,7 +85,7 @@ storage systems. Because the protocols have changed in different versions of Hadoop, you must build Spark against the same version that your cluster runs. Please refer to the build documentation at -["Specifying the Hadoop Version"](http://spark.apache.org/docs/latest/building-with-maven.html#specifying-the-hadoop-version) +["Specifying the Hadoop Version"](http://spark.apache.org/docs/latest/building-spark.html#specifying-the-hadoop-version) for detailed guidance on building for a particular distribution of Hadoop, including building for particular Hive and Hive Thriftserver distributions. See also ["Third Party Hadoop Distributions"](http://spark.apache.org/docs/latest/hadoop-third-party-distributions.html) diff --git a/assembly/pom.xml b/assembly/pom.xml index cbf5b6c4aa8df..d3bb4bde0c412 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -114,16 +114,6 @@ META-INF/*.RSA - - - org.jblas:jblas - - - lib/static/Linux/i386/** - lib/static/Mac OS X/** - lib/static/Windows/** - - diff --git a/bin/pyspark b/bin/pyspark index e7f6a1a072c2a..776b28dc41099 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -89,7 +89,6 @@ export PYTHONSTARTUP="$SPARK_HOME/python/pyspark/shell.py" if [[ -n "$SPARK_TESTING" ]]; then unset YARN_CONF_DIR unset HADOOP_CONF_DIR - export PYSPARK_SUBMIT_ARGS=pyspark-shell if [[ -n "$PYSPARK_DOC_TEST" ]]; then exec "$PYSPARK_DRIVER_PYTHON" -m doctest $1 else diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 0886b0276fb90..67f81d33361e1 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -15,7 +15,7 @@ # - SPARK_PUBLIC_DNS, to set the public DNS name of the driver program # - SPARK_CLASSPATH, default classpath entries to append # - SPARK_LOCAL_DIRS, storage directories to use on this node for shuffle and RDD data -# - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos +# - MESOS_NATIVE_JAVA_LIBRARY, to point to your libmesos.so if you use Mesos # Options read in YARN client mode # - HADOOP_CONF_DIR, to point Spark towards Hadoop configuration files diff --git a/core/pom.xml b/core/pom.xml index 4164a3a7208d4..81f8cba711df6 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -414,7 +414,7 @@ true true - guava,jetty-io,jetty-servlet,jetty-continuation,jetty-http,jetty-plus,jetty-util,jetty-server + guava,jetty-io,jetty-servlet,jetty-continuation,jetty-http,jetty-plus,jetty-util,jetty-server,jetty-security true diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 0c59a61e81393..9b05c9623b704 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -145,7 +145,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Keep cleaning RDD, shuffle, and broadcast state. */ - private def keepCleaning(): Unit = Utils.logUncaughtExceptions { + private def keepCleaning(): Unit = Utils.tryOrStopSparkContext(sc) { while (!stopped) { try { val reference = Option(referenceQueue.remove(ContextCleaner.REF_QUEUE_POLL_TIMEOUT)) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1a0bee4e3aea9..228ff715fe7cb 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1093,7 +1093,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli def addFile(path: String, recursive: Boolean): Unit = { val uri = new URI(path) val schemeCorrectedPath = uri.getScheme match { - case null | "local" => "file:" + uri.getPath + case null | "local" => new File(path).getCanonicalFile.toURI.toString case _ => path } @@ -1104,7 +1104,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli if (!fs.exists(hadoopPath)) { throw new FileNotFoundException(s"Added file $hadoopPath does not exist.") } - val isDir = fs.isDirectory(hadoopPath) + val isDir = fs.getFileStatus(hadoopPath).isDir if (!isLocal && scheme == "file" && isDir) { throw new SparkException(s"addFile does not support local directories when not running " + "local mode.") @@ -1736,7 +1736,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } } - listenerBus.start() + listenerBus.start(this) } /** Post the application start event */ diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 29a5cd5fdac76..48fd3e7e23d52 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -151,11 +151,7 @@ case object TaskKilled extends TaskFailedReason { * Task requested the driver to commit, but was denied. */ @DeveloperApi -case class TaskCommitDenied( - jobID: Int, - partitionID: Int, - attemptID: Int) - extends TaskFailedReason { +case class TaskCommitDenied(jobID: Int, partitionID: Int, attemptID: Int) extends TaskFailedReason { override def toErrorString: String = s"TaskCommitDenied (Driver denied task commit)" + s" for job: $jobID, partition: $partitionID, attempt: $attemptID" } diff --git a/core/src/main/scala/org/apache/spark/TaskState.scala b/core/src/main/scala/org/apache/spark/TaskState.scala index 0bf1e4a5e2ccd..d85a6d683427d 100644 --- a/core/src/main/scala/org/apache/spark/TaskState.scala +++ b/core/src/main/scala/org/apache/spark/TaskState.scala @@ -46,5 +46,6 @@ private[spark] object TaskState extends Enumeration { case MesosTaskState.TASK_FAILED => FAILED case MesosTaskState.TASK_KILLED => KILLED case MesosTaskState.TASK_LOST => LOST + case MesosTaskState.TASK_ERROR => LOST } } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index 8e8f7f6c4fda2..79e4ebf2db578 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -32,7 +32,8 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.StatCounter import org.apache.spark.util.Utils -class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, JavaDoubleRDD] { +class JavaDoubleRDD(val srdd: RDD[scala.Double]) + extends AbstractJavaRDDLike[JDouble, JavaDoubleRDD] { override val classTag: ClassTag[JDouble] = implicitly[ClassTag[JDouble]] diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 7af3538262fd6..a023712be1166 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -39,12 +39,13 @@ import org.apache.spark.api.java.function.{Function => JFunction, Function2 => J import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.{OrderedRDDFunctions, RDD} import org.apache.spark.rdd.RDD.rddToPairRDDFunctions +import org.apache.spark.serializer.Serializer import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) (implicit val kClassTag: ClassTag[K], implicit val vClassTag: ClassTag[V]) - extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] { + extends AbstractJavaRDDLike[(K, V), JavaPairRDD[K, V]] { override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd) @@ -227,24 +228,51 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) * - `mergeCombiners`, to combine two C's into a single one. * - * In addition, users can control the partitioning of the output RDD, and whether to perform - * map-side aggregation (if a mapper can produce multiple items with the same key). + * In addition, users can control the partitioning of the output RDD, the serializer that is use + * for the shuffle, and whether to perform map-side aggregation (if a mapper can produce multiple + * items with the same key). */ def combineByKey[C](createCombiner: JFunction[V, C], - mergeValue: JFunction2[C, V, C], - mergeCombiners: JFunction2[C, C, C], - partitioner: Partitioner): JavaPairRDD[K, C] = { - implicit val ctag: ClassTag[C] = fakeClassTag + mergeValue: JFunction2[C, V, C], + mergeCombiners: JFunction2[C, C, C], + partitioner: Partitioner, + mapSideCombine: Boolean, + serializer: Serializer): JavaPairRDD[K, C] = { + implicit val ctag: ClassTag[C] = fakeClassTag fromRDD(rdd.combineByKey( createCombiner, mergeValue, mergeCombiners, - partitioner + partitioner, + mapSideCombine, + serializer )) } /** - * Simplified version of combineByKey that hash-partitions the output RDD. + * Generic function to combine the elements for each key using a custom set of aggregation + * functions. Turns a JavaPairRDD[(K, V)] into a result of type JavaPairRDD[(K, C)], for a + * "combined type" C * Note that V and C can be different -- for example, one might group an + * RDD of type (Int, Int) into an RDD of type (Int, List[Int]). Users provide three + * functions: + * + * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) + * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) + * - `mergeCombiners`, to combine two C's into a single one. + * + * In addition, users can control the partitioning of the output RDD. This method automatically + * uses map-side aggregation in shuffling the RDD. + */ + def combineByKey[C](createCombiner: JFunction[V, C], + mergeValue: JFunction2[C, V, C], + mergeCombiners: JFunction2[C, C, C], + partitioner: Partitioner): JavaPairRDD[K, C] = { + combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner, true, null) + } + + /** + * Simplified version of combineByKey that hash-partitions the output RDD and uses map-side + * aggregation. */ def combineByKey[C](createCombiner: JFunction[V, C], mergeValue: JFunction2[C, V, C], @@ -488,7 +516,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Simplified version of combineByKey that hash-partitions the resulting RDD using the existing - * partitioner/parallelism level. + * partitioner/parallelism level and using map-side aggregation. */ def combineByKey[C](createCombiner: JFunction[V, C], mergeValue: JFunction2[C, V, C], diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 86fb374bef1e3..645dc3bfb6b06 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -30,7 +30,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) - extends JavaRDDLike[T, JavaRDD[T]] { + extends AbstractJavaRDDLike[T, JavaRDD[T]] { override def wrapRDD(rdd: RDD[T]): JavaRDD[T] = JavaRDD.fromRDD(rdd) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 0f91c942ecd50..8da42934a7d96 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -38,6 +38,14 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils +/** + * As a workaround for https://issues.scala-lang.org/browse/SI-8905, implementations + * of JavaRDDLike should extend this dummy abstract class instead of directly inheriting + * from the trait. See SPARK-3266 for additional details. + */ +private[spark] abstract class AbstractJavaRDDLike[T, This <: JavaRDDLike[T, This]] + extends JavaRDDLike[T, This] + /** * Defines operations common to several Java RDD implementations. * Note that this trait is not intended to be implemented by user code. 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 8d4a53b4ca9b0..4c71b69069eb3 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 @@ -76,7 +76,6 @@ private[spark] class PythonRDD( context.addTaskCompletionListener { context => writerThread.shutdownOnTaskCompletion() - writerThread.join() if (!reuse_worker || !released) { try { worker.close() @@ -248,13 +247,17 @@ private[spark] class PythonRDD( } catch { case e: Exception if context.isCompleted || context.isInterrupted => logDebug("Exception thrown after task completion (likely due to cleanup)", e) - Utils.tryLog(worker.shutdownOutput()) + if (!worker.isClosed) { + Utils.tryLog(worker.shutdownOutput()) + } case e: Exception => // We must avoid throwing exceptions here, because the thread uncaught exception handler // will kill the whole executor (see org.apache.spark.executor.Executor). _exception = e - Utils.tryLog(worker.shutdownOutput()) + if (!worker.isClosed) { + Utils.tryLog(worker.shutdownOutput()) + } } finally { // Release memory used by this thread for shuffles env.shuffleMemoryManager.releaseMemoryForThisThread() diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 415bd50591692..53bc62aff7395 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.{IntParam, MemoryParam} /** * Command-line parser for the driver client. */ -private[spark] class ClientArguments(args: Array[String]) { +private[deploy] class ClientArguments(args: Array[String]) { import ClientArguments._ var cmd: String = "" // 'launch' or 'kill' @@ -96,7 +96,7 @@ private[spark] class ClientArguments(args: Array[String]) { /** * Print usage and exit JVM with the given exit code. */ - def printUsageAndExit(exitCode: Int) { + private def printUsageAndExit(exitCode: Int) { // TODO: It wouldn't be too hard to allow users to submit their app and dependency jars // separately similar to in the YARN client. val usage = @@ -116,10 +116,10 @@ private[spark] class ClientArguments(args: Array[String]) { } } -object ClientArguments { - private[spark] val DEFAULT_CORES = 1 - private[spark] val DEFAULT_MEMORY = 512 // MB - private[spark] val DEFAULT_SUPERVISE = false +private[deploy] object ClientArguments { + val DEFAULT_CORES = 1 + val DEFAULT_MEMORY = 512 // MB + val DEFAULT_SUPERVISE = false def isValidJarUrl(s: String): Boolean = { try { diff --git a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala index b056a19ce6598..659fb434a80f5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -private[spark] class DriverDescription( +private[deploy] class DriverDescription( val jarUrl: String, val mem: Int, val cores: Int, diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala index 2abf0b69dddb3..ec23371b52f93 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala @@ -22,7 +22,7 @@ package org.apache.spark.deploy * This state is sufficient for the Master to reconstruct its internal data structures during * failover. */ -private[spark] class ExecutorDescription( +private[deploy] class ExecutorDescription( val appId: String, val execId: Int, val cores: Int, diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala index 9f34d01e6db48..efa88c62e1f5d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -private[spark] object ExecutorState extends Enumeration { +private[deploy] object ExecutorState extends Enumeration { val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST, EXITED = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index 47dbcd87c35b5..4e58aa0ed4c7e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -55,29 +55,29 @@ import org.apache.spark.deploy.master.{RecoveryState, SparkCuratorUtil} * - The docker images tagged spark-test-master and spark-test-worker are built from the * docker/ directory. Run 'docker/spark-test/build' to generate these. */ -private[spark] object FaultToleranceTest extends App with Logging { +private object FaultToleranceTest extends App with Logging { - val conf = new SparkConf() - val ZK_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + private val conf = new SparkConf() + private val ZK_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") - val masters = ListBuffer[TestMasterInfo]() - val workers = ListBuffer[TestWorkerInfo]() - var sc: SparkContext = _ + private val masters = ListBuffer[TestMasterInfo]() + private val workers = ListBuffer[TestWorkerInfo]() + private var sc: SparkContext = _ - val zk = SparkCuratorUtil.newClient(conf) + private val zk = SparkCuratorUtil.newClient(conf) - var numPassed = 0 - var numFailed = 0 + private var numPassed = 0 + private var numFailed = 0 - val sparkHome = System.getenv("SPARK_HOME") + private val sparkHome = System.getenv("SPARK_HOME") assertTrue(sparkHome != null, "Run with a valid SPARK_HOME") - val containerSparkHome = "/opt/spark" - val dockerMountDir = "%s:%s".format(sparkHome, containerSparkHome) + private val containerSparkHome = "/opt/spark" + private val dockerMountDir = "%s:%s".format(sparkHome, containerSparkHome) System.setProperty("spark.driver.host", "172.17.42.1") // default docker host ip - def afterEach() { + private def afterEach() { if (sc != null) { sc.stop() sc = null @@ -179,7 +179,7 @@ private[spark] object FaultToleranceTest extends App with Logging { } } - def test(name: String)(fn: => Unit) { + private def test(name: String)(fn: => Unit) { try { fn numPassed += 1 @@ -197,19 +197,19 @@ private[spark] object FaultToleranceTest extends App with Logging { afterEach() } - def addMasters(num: Int) { + private def addMasters(num: Int) { logInfo(s">>>>> ADD MASTERS $num <<<<<") (1 to num).foreach { _ => masters += SparkDocker.startMaster(dockerMountDir) } } - def addWorkers(num: Int) { + private def addWorkers(num: Int) { logInfo(s">>>>> ADD WORKERS $num <<<<<") val masterUrls = getMasterUrls(masters) (1 to num).foreach { _ => workers += SparkDocker.startWorker(dockerMountDir, masterUrls) } } /** Creates a SparkContext, which constructs a Client to interact with our cluster. */ - def createClient() = { + private def createClient() = { logInfo(">>>>> CREATE CLIENT <<<<<") if (sc != null) { sc.stop() } // Counter-hack: Because of a hack in SparkEnv#create() that changes this @@ -218,17 +218,17 @@ private[spark] object FaultToleranceTest extends App with Logging { sc = new SparkContext(getMasterUrls(masters), "fault-tolerance", containerSparkHome) } - def getMasterUrls(masters: Seq[TestMasterInfo]): String = { + private def getMasterUrls(masters: Seq[TestMasterInfo]): String = { "spark://" + masters.map(master => master.ip + ":7077").mkString(",") } - def getLeader: TestMasterInfo = { + private def getLeader: TestMasterInfo = { val leaders = masters.filter(_.state == RecoveryState.ALIVE) assertTrue(leaders.size == 1) leaders(0) } - def killLeader(): Unit = { + private def killLeader(): Unit = { logInfo(">>>>> KILL LEADER <<<<<") masters.foreach(_.readState()) val leader = getLeader @@ -236,9 +236,9 @@ private[spark] object FaultToleranceTest extends App with Logging { leader.kill() } - def delay(secs: Duration = 5.seconds) = Thread.sleep(secs.toMillis) + private def delay(secs: Duration = 5.seconds) = Thread.sleep(secs.toMillis) - def terminateCluster() { + private def terminateCluster() { logInfo(">>>>> TERMINATE CLUSTER <<<<<") masters.foreach(_.kill()) workers.foreach(_.kill()) @@ -247,7 +247,7 @@ private[spark] object FaultToleranceTest extends App with Logging { } /** This includes Client retry logic, so it may take a while if the cluster is recovering. */ - def assertUsable() = { + private def assertUsable() = { val f = future { try { val res = sc.parallelize(0 until 10).collect() @@ -269,7 +269,7 @@ private[spark] object FaultToleranceTest extends App with Logging { * Asserts that the cluster is usable and that the expected masters and workers * are all alive in a proper configuration (e.g., only one leader). */ - def assertValidClusterState() = { + private def assertValidClusterState() = { logInfo(">>>>> ASSERT VALID CLUSTER STATE <<<<<") assertUsable() var numAlive = 0 @@ -325,7 +325,7 @@ private[spark] object FaultToleranceTest extends App with Logging { } } - def assertTrue(bool: Boolean, message: String = "") { + private def assertTrue(bool: Boolean, message: String = "") { if (!bool) { throw new IllegalStateException("Assertion failed: " + message) } @@ -335,7 +335,7 @@ private[spark] object FaultToleranceTest extends App with Logging { numFailed)) } -private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile: File) +private class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile: File) extends Logging { implicit val formats = org.json4s.DefaultFormats @@ -377,7 +377,7 @@ private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val format(ip, dockerId.id, logFile.getAbsolutePath, state) } -private[spark] class TestWorkerInfo(val ip: String, val dockerId: DockerId, val logFile: File) +private class TestWorkerInfo(val ip: String, val dockerId: DockerId, val logFile: File) extends Logging { implicit val formats = org.json4s.DefaultFormats @@ -390,7 +390,7 @@ private[spark] class TestWorkerInfo(val ip: String, val dockerId: DockerId, val "[ip=%s, id=%s, logFile=%s]".format(ip, dockerId, logFile.getAbsolutePath) } -private[spark] object SparkDocker { +private object SparkDocker { def startMaster(mountDir: String): TestMasterInfo = { val cmd = Docker.makeRunCmd("spark-test-master", mountDir = mountDir) val (ip, id, outFile) = startNode(cmd) @@ -425,11 +425,11 @@ private[spark] object SparkDocker { } } -private[spark] class DockerId(val id: String) { +private class DockerId(val id: String) { override def toString = id } -private[spark] object Docker extends Logging { +private object Docker extends Logging { def makeRunCmd(imageTag: String, args: String = "", mountDir: String = ""): ProcessBuilder = { val mountCmd = if (mountDir != "") { " -v " + mountDir } else "" diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index 696f32a6f5730..458a7c3a455de 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -23,7 +23,7 @@ import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateR import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} import org.apache.spark.deploy.worker.ExecutorRunner -private[spark] object JsonProtocol { +private[deploy] object JsonProtocol { def writeWorkerInfo(obj: WorkerInfo) = { ("id" -> obj.id) ~ ("host" -> obj.host) ~ 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 4a74641f4e1fa..4f506be63fe59 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -45,7 +45,7 @@ import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, U * Whether to submit, kill, or request the status of an application. * The latter two operations are currently supported only for standalone cluster mode. */ -private[spark] object SparkSubmitAction extends Enumeration { +private[deploy] object SparkSubmitAction extends Enumeration { type SparkSubmitAction = Value val SUBMIT, KILL, REQUEST_STATUS = Value } @@ -137,7 +137,7 @@ object SparkSubmit { * Second, we use this launch environment to invoke the main method of the child * main class. */ - private[spark] def submit(args: SparkSubmitArguments): Unit = { + private def submit(args: SparkSubmitArguments): Unit = { val (childArgs, childClasspath, sysProps, childMainClass) = prepareSubmitEnvironment(args) def doRunMain(): Unit = { @@ -199,7 +199,7 @@ object SparkSubmit { * (4) the main class for the child * Exposed for testing. */ - private[spark] def prepareSubmitEnvironment(args: SparkSubmitArguments) + private[deploy] def prepareSubmitEnvironment(args: SparkSubmitArguments) : (Seq[String], Seq[String], Map[String, String], String) = { // Return values val childArgs = new ArrayBuffer[String]() @@ -598,32 +598,32 @@ object SparkSubmit { /** * Return whether the given primary resource represents a shell. */ - private[spark] def isShell(primaryResource: String): Boolean = { + private[deploy] def isShell(primaryResource: String): Boolean = { primaryResource == SPARK_SHELL || primaryResource == PYSPARK_SHELL } /** * Return whether the given main class represents a sql shell. */ - private[spark] def isSqlShell(mainClass: String): Boolean = { + private def isSqlShell(mainClass: String): Boolean = { mainClass == "org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" } /** * Return whether the given main class represents a thrift server. */ - private[spark] def isThriftServer(mainClass: String): Boolean = { + private def isThriftServer(mainClass: String): Boolean = { mainClass == "org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" } /** * Return whether the given primary resource requires running python. */ - private[spark] def isPython(primaryResource: String): Boolean = { + private[deploy] def isPython(primaryResource: String): Boolean = { primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL } - private[spark] def isInternal(primaryResource: String): Boolean = { + private[deploy] def isInternal(primaryResource: String): Boolean = { primaryResource == SPARK_INTERNAL } @@ -631,7 +631,7 @@ object SparkSubmit { * Merge a sequence of comma-separated file lists, some of which may be null to indicate * no files, into a single comma-separated string. */ - private[spark] def mergeFileLists(lists: String*): String = { + private def mergeFileLists(lists: String*): String = { val merged = lists.filter(_ != null) .flatMap(_.split(",")) .mkString(",") @@ -640,10 +640,10 @@ object SparkSubmit { } /** Provides utility functions to be used inside SparkSubmit. */ -private[spark] object SparkSubmitUtils { +private[deploy] object SparkSubmitUtils { // Exposed for testing - private[spark] var printStream = SparkSubmit.printStream + var printStream = SparkSubmit.printStream /** * Represents a Maven Coordinate @@ -651,7 +651,7 @@ private[spark] object SparkSubmitUtils { * @param artifactId the artifactId of the coordinate * @param version the version of the coordinate */ - private[spark] case class MavenCoordinate(groupId: String, artifactId: String, version: String) + private[deploy] case class MavenCoordinate(groupId: String, artifactId: String, version: String) /** * Extracts maven coordinates from a comma-delimited string. Coordinates should be provided @@ -659,7 +659,7 @@ private[spark] object SparkSubmitUtils { * @param coordinates Comma-delimited string of maven coordinates * @return Sequence of Maven coordinates */ - private[spark] def extractMavenCoordinates(coordinates: String): Seq[MavenCoordinate] = { + def extractMavenCoordinates(coordinates: String): Seq[MavenCoordinate] = { coordinates.split(",").map { p => val splits = p.replace("/", ":").split(":") require(splits.length == 3, s"Provided Maven Coordinates must be in the form " + @@ -679,7 +679,7 @@ private[spark] object SparkSubmitUtils { * @param remoteRepos Comma-delimited string of remote repositories * @return A ChainResolver used by Ivy to search for and resolve dependencies. */ - private[spark] def createRepoResolvers(remoteRepos: Option[String]): ChainResolver = { + def createRepoResolvers(remoteRepos: Option[String]): ChainResolver = { // We need a chain resolver if we want to check multiple repositories val cr = new ChainResolver cr.setName("list") @@ -722,7 +722,7 @@ private[spark] object SparkSubmitUtils { * @param cacheDirectory directory where jars are cached * @return a comma-delimited list of paths for the dependencies */ - private[spark] def resolveDependencyPaths( + def resolveDependencyPaths( artifacts: Array[AnyRef], cacheDirectory: File): String = { artifacts.map { artifactInfo => @@ -734,7 +734,7 @@ private[spark] object SparkSubmitUtils { } /** Adds the given maven coordinates to Ivy's module descriptor. */ - private[spark] def addDependenciesToIvy( + def addDependenciesToIvy( md: DefaultModuleDescriptor, artifacts: Seq[MavenCoordinate], ivyConfName: String): Unit = { @@ -748,7 +748,7 @@ private[spark] object SparkSubmitUtils { } /** Add exclusion rules for dependencies already included in the spark-assembly */ - private[spark] def addExclusionRules( + def addExclusionRules( ivySettings: IvySettings, ivyConfName: String, md: DefaultModuleDescriptor): Unit = { @@ -777,7 +777,7 @@ private[spark] object SparkSubmitUtils { } /** A nice function to use in tests as well. Values are dummy strings. */ - private[spark] def getModuleDescriptor = DefaultModuleDescriptor.newDefaultInstance( + def getModuleDescriptor = DefaultModuleDescriptor.newDefaultInstance( ModuleRevisionId.newInstance("org.apache.spark", "spark-submit-parent", "1.0")) /** @@ -788,7 +788,7 @@ private[spark] object SparkSubmitUtils { * @return The comma-delimited path to the jars of the given maven artifacts including their * transitive dependencies */ - private[spark] def resolveMavenCoordinates( + def resolveMavenCoordinates( coordinates: String, remoteRepos: Option[String], ivyPath: Option[String], 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 94e4bdbfb7d7b..2250d5a28e4ef 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils * Parses and encapsulates arguments from the spark-submit script. * The env argument is used for testing. */ -private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, String] = sys.env) +private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, String] = sys.env) extends SparkSubmitArgumentsParser { var master: String = null var deployMode: String = null diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index ffe940fbda2fb..2d24083a77b73 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -47,18 +47,18 @@ private[spark] class AppClient( conf: SparkConf) extends Logging { - val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl(_, AkkaUtils.protocol(actorSystem))) + private val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl(_, AkkaUtils.protocol(actorSystem))) - val REGISTRATION_TIMEOUT = 20.seconds - val REGISTRATION_RETRIES = 3 + private val REGISTRATION_TIMEOUT = 20.seconds + private val REGISTRATION_RETRIES = 3 - var masterAddress: Address = null - var actor: ActorRef = null - var appId: String = null - var registered = false - var activeMasterUrl: String = null + private var masterAddress: Address = null + private var actor: ActorRef = null + private var appId: String = null + private var registered = false + private var activeMasterUrl: String = null - class ClientActor extends Actor with ActorLogReceive with Logging { + private class ClientActor extends Actor with ActorLogReceive with Logging { var master: ActorSelection = null var alreadyDisconnected = false // To avoid calling listener.disconnected() multiple times var alreadyDead = false // To avoid calling listener.dead() multiple times diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 88a0862b96afe..c1c4812f17fbe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -23,7 +23,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} private[spark] object TestClient { - class TestListener extends AppClientListener with Logging { + private class TestListener extends AppClientListener with Logging { def connected(id: String) { logInfo("Connected to master, got app ID " + id) } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index 553bf3cb945ab..ea6c85ee511d5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.history import org.apache.spark.ui.SparkUI -private[spark] case class ApplicationHistoryInfo( +private[history] case class ApplicationHistoryInfo( id: String, name: String, startTime: Long, @@ -28,7 +28,7 @@ private[spark] case class ApplicationHistoryInfo( sparkUser: String, completed: Boolean = false) -private[spark] abstract class ApplicationHistoryProvider { +private[history] abstract class ApplicationHistoryProvider { /** * Returns a list of applications available for the history server to show. diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 16d88c17d1a76..7fde02040927d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -93,7 +93,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis */ private def getRunner(operateFun: () => Unit): Runnable = { new Runnable() { - override def run() = Utils.logUncaughtExceptions { + override def run() = Utils.tryOrExit { operateFun() } } 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 26ebc75971c66..6e432d63c6b5a 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 @@ -23,7 +23,7 @@ import scala.xml.Node import org.apache.spark.ui.{WebUIPage, UIUtils} -private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { +private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { private val pageSize = 20 private val plusOrMinus = 2 diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index b1270ade9f750..a2a97a7877ce7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -23,7 +23,8 @@ import org.apache.spark.util.Utils /** * Command-line parser for the master. */ -private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String]) extends Logging { +private[history] class HistoryServerArguments(conf: SparkConf, args: Array[String]) + extends Logging { private var propertiesFile: String = null parse(args.toList) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index a962dc4af2f6c..536aedb6f9fe9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -28,7 +28,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.ApplicationDescription import org.apache.spark.util.Utils -private[spark] class ApplicationInfo( +private[deploy] class ApplicationInfo( val startTime: Long, val id: String, val desc: ApplicationDescription, @@ -75,14 +75,15 @@ private[spark] class ApplicationInfo( } } - def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): ExecutorDesc = { + private[master] def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): + ExecutorDesc = { val exec = new ExecutorDesc(newExecutorId(useID), this, worker, cores, desc.memoryPerSlave) executors(exec.id) = exec coresGranted += cores exec } - def removeExecutor(exec: ExecutorDesc) { + private[master] def removeExecutor(exec: ExecutorDesc) { if (executors.contains(exec.id)) { removedExecutors += executors(exec.id) executors -= exec.id @@ -90,22 +91,22 @@ private[spark] class ApplicationInfo( } } - val requestedCores = desc.maxCores.getOrElse(defaultCores) + private[master] val requestedCores = desc.maxCores.getOrElse(defaultCores) - def coresLeft: Int = requestedCores - coresGranted + private[master] def coresLeft: Int = requestedCores - coresGranted private var _retryCount = 0 - def retryCount = _retryCount + private[master] def retryCount = _retryCount - def incrementRetryCount() = { + private[master] def incrementRetryCount() = { _retryCount += 1 _retryCount } - def resetRetryCount() = _retryCount = 0 + private[master] def resetRetryCount() = _retryCount = 0 - def markFinished(endState: ApplicationState.Value) { + private[master] def markFinished(endState: ApplicationState.Value) { state = endState endTime = System.currentTimeMillis() } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala index 38db02cd2421b..017e8b55cbe7f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala @@ -21,7 +21,7 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.metrics.source.Source -class ApplicationSource(val application: ApplicationInfo) extends Source { +private[master] class ApplicationSource(val application: ApplicationInfo) extends Source { override val metricRegistry = new MetricRegistry() override val sourceName = "%s.%s.%s".format("application", application.desc.name, System.currentTimeMillis()) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala index f5b946329ae9b..37bfcdfdf4777 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object ApplicationState extends Enumeration { +private[master] object ApplicationState extends Enumeration { type ApplicationState = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala index 9d3d7938c6ccb..b197dbcbfe294 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala @@ -23,7 +23,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.DriverDescription import org.apache.spark.util.Utils -private[spark] class DriverInfo( +private[deploy] class DriverInfo( val startTime: Long, val id: String, val desc: DriverDescription, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala index 26a68bade3c60..35ff33a61653c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object DriverState extends Enumeration { +private[deploy] object DriverState extends Enumeration { type DriverState = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala index 5d620dfcabad5..fc62b094def67 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.master import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} -private[spark] class ExecutorDesc( +private[master] class ExecutorDesc( val id: Int, val application: ApplicationInfo, val worker: WorkerInfo, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala index 36a2e2c6a6349..d2d30bfd7fcba 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -33,7 +33,7 @@ import org.apache.spark.Logging * @param dir Directory to store files. Created if non-existent (but not recursively). * @param serialization Used to serialize our objects. */ -private[spark] class FileSystemPersistenceEngine( +private[master] class FileSystemPersistenceEngine( val dir: String, val serialization: Serialization) extends PersistenceEngine with Logging { 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 15814293227ab..1b42121c8db05 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 @@ -49,7 +49,7 @@ import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} -private[spark] class Master( +private[master] class Master( host: String, port: Int, webUiPort: Int, @@ -59,65 +59,68 @@ private[spark] class Master( import context.dispatcher // to use Akka's scheduler.schedule() - val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs - val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000 - val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) - val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) - val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) - val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs + + private val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000 + private val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) + private val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) + private val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) + private val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") val workers = new HashSet[WorkerInfo] - val idToWorker = new HashMap[String, WorkerInfo] - val addressToWorker = new HashMap[Address, WorkerInfo] - - val apps = new HashSet[ApplicationInfo] val idToApp = new HashMap[String, ApplicationInfo] - val actorToApp = new HashMap[ActorRef, ApplicationInfo] - val addressToApp = new HashMap[Address, ApplicationInfo] val waitingApps = new ArrayBuffer[ApplicationInfo] - val completedApps = new ArrayBuffer[ApplicationInfo] - var nextAppNumber = 0 - val appIdToUI = new HashMap[String, SparkUI] + val apps = new HashSet[ApplicationInfo] + + private val idToWorker = new HashMap[String, WorkerInfo] + private val addressToWorker = new HashMap[Address, WorkerInfo] + + private val actorToApp = new HashMap[ActorRef, ApplicationInfo] + private val addressToApp = new HashMap[Address, ApplicationInfo] + private val completedApps = new ArrayBuffer[ApplicationInfo] + private var nextAppNumber = 0 + private val appIdToUI = new HashMap[String, SparkUI] - val drivers = new HashSet[DriverInfo] - val completedDrivers = new ArrayBuffer[DriverInfo] - val waitingDrivers = new ArrayBuffer[DriverInfo] // Drivers currently spooled for scheduling - var nextDriverNumber = 0 + private val drivers = new HashSet[DriverInfo] + private val completedDrivers = new ArrayBuffer[DriverInfo] + // Drivers currently spooled for scheduling + private val waitingDrivers = new ArrayBuffer[DriverInfo] + private var nextDriverNumber = 0 Utils.checkHost(host, "Expected hostname") - val masterMetricsSystem = MetricsSystem.createMetricsSystem("master", conf, securityMgr) - val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications", conf, + private val masterMetricsSystem = MetricsSystem.createMetricsSystem("master", conf, securityMgr) + private val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications", conf, securityMgr) - val masterSource = new MasterSource(this) + private val masterSource = new MasterSource(this) - val webUi = new MasterWebUI(this, webUiPort) + private val webUi = new MasterWebUI(this, webUiPort) - val masterPublicAddress = { + private val masterPublicAddress = { val envVar = conf.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host } - val masterUrl = "spark://" + host + ":" + port - var masterWebUiUrl: String = _ + private val masterUrl = "spark://" + host + ":" + port + private var masterWebUiUrl: String = _ - var state = RecoveryState.STANDBY + private var state = RecoveryState.STANDBY - var persistenceEngine: PersistenceEngine = _ + private var persistenceEngine: PersistenceEngine = _ - var leaderElectionAgent: LeaderElectionAgent = _ + private var leaderElectionAgent: LeaderElectionAgent = _ private var recoveryCompletionTask: Cancellable = _ // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - val spreadOutApps = conf.getBoolean("spark.deploy.spreadOut", true) + private val spreadOutApps = conf.getBoolean("spark.deploy.spreadOut", true) // Default maxCores for applications that don't specify it (i.e. pass Int.MaxValue) - val defaultCores = conf.getInt("spark.deploy.defaultCores", Int.MaxValue) + private val defaultCores = conf.getInt("spark.deploy.defaultCores", Int.MaxValue) if (defaultCores < 1) { throw new SparkException("spark.deploy.defaultCores must be positive") } @@ -449,11 +452,11 @@ private[spark] class Master( } } - def canCompleteRecovery = + private def canCompleteRecovery = workers.count(_.state == WorkerState.UNKNOWN) == 0 && apps.count(_.state == ApplicationState.UNKNOWN) == 0 - def beginRecovery(storedApps: Seq[ApplicationInfo], storedDrivers: Seq[DriverInfo], + private def beginRecovery(storedApps: Seq[ApplicationInfo], storedDrivers: Seq[DriverInfo], storedWorkers: Seq[WorkerInfo]) { for (app <- storedApps) { logInfo("Trying to recover app: " + app.id) @@ -484,7 +487,7 @@ private[spark] class Master( } } - def completeRecovery() { + private def completeRecovery() { // Ensure "only-once" recovery semantics using a short synchronization period. synchronized { if (state != RecoveryState.RECOVERING) { return } @@ -517,7 +520,7 @@ private[spark] class Master( * launched an executor for the app on it (right now the standalone backend doesn't like having * two executors on the same worker). */ - def canUse(app: ApplicationInfo, worker: WorkerInfo): Boolean = { + private def canUse(app: ApplicationInfo, worker: WorkerInfo): Boolean = { worker.memoryFree >= app.desc.memoryPerSlave && !worker.hasExecutor(app) } @@ -596,7 +599,7 @@ private[spark] class Master( } } - def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc) { + private def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc) { logInfo("Launching executor " + exec.fullId + " on worker " + worker.id) worker.addExecutor(exec) worker.actor ! LaunchExecutor(masterUrl, @@ -605,7 +608,7 @@ private[spark] class Master( exec.id, worker.id, worker.hostPort, exec.cores, exec.memory) } - def registerWorker(worker: WorkerInfo): Boolean = { + private def registerWorker(worker: WorkerInfo): Boolean = { // There may be one or more refs to dead workers on this same node (w/ different ID's), // remove them. workers.filter { w => @@ -633,7 +636,7 @@ private[spark] class Master( true } - def removeWorker(worker: WorkerInfo) { + private def removeWorker(worker: WorkerInfo) { logInfo("Removing worker " + worker.id + " on " + worker.host + ":" + worker.port) worker.setState(WorkerState.DEAD) idToWorker -= worker.id @@ -656,20 +659,20 @@ private[spark] class Master( persistenceEngine.removeWorker(worker) } - def relaunchDriver(driver: DriverInfo) { + private def relaunchDriver(driver: DriverInfo) { driver.worker = None driver.state = DriverState.RELAUNCHING waitingDrivers += driver schedule() } - def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { + private def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { val now = System.currentTimeMillis() val date = new Date(now) new ApplicationInfo(now, newApplicationId(date), desc, date, driver, defaultCores) } - def registerApplication(app: ApplicationInfo): Unit = { + private def registerApplication(app: ApplicationInfo): Unit = { val appAddress = app.driver.path.address if (addressToApp.contains(appAddress)) { logInfo("Attempted to re-register application at same address: " + appAddress) @@ -684,7 +687,7 @@ private[spark] class Master( waitingApps += app } - def finishApplication(app: ApplicationInfo) { + private def finishApplication(app: ApplicationInfo) { removeApplication(app, ApplicationState.FINISHED) } @@ -732,7 +735,7 @@ private[spark] class Master( * Rebuild a new SparkUI from the given application's event logs. * Return whether this is successful. */ - def rebuildSparkUI(app: ApplicationInfo): Boolean = { + private def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" try { @@ -764,8 +767,9 @@ private[spark] class Master( val replayBus = new ReplayListenerBus() val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), appName + status, HistoryServer.UI_PATH_PREFIX + s"/${app.id}") + val maybeTruncated = eventLogFile.endsWith(EventLoggingListener.IN_PROGRESS) try { - replayBus.replay(logInput, eventLogFile) + replayBus.replay(logInput, eventLogFile, maybeTruncated) } finally { logInput.close() } @@ -797,14 +801,14 @@ private[spark] class Master( } /** Generate a new app ID given a app's submission date */ - def newApplicationId(submitDate: Date): String = { + private def newApplicationId(submitDate: Date): String = { val appId = "app-%s-%04d".format(createDateFormat.format(submitDate), nextAppNumber) nextAppNumber += 1 appId } /** Check for, and remove, any timed-out workers */ - def timeOutDeadWorkers() { + private def timeOutDeadWorkers() { // Copy the workers into an array so we don't modify the hashset while iterating through it val currentTime = System.currentTimeMillis() val toRemove = workers.filter(_.lastHeartbeat < currentTime - WORKER_TIMEOUT).toArray @@ -821,19 +825,19 @@ private[spark] class Master( } } - def newDriverId(submitDate: Date): String = { + private def newDriverId(submitDate: Date): String = { val appId = "driver-%s-%04d".format(createDateFormat.format(submitDate), nextDriverNumber) nextDriverNumber += 1 appId } - def createDriver(desc: DriverDescription): DriverInfo = { + private def createDriver(desc: DriverDescription): DriverInfo = { val now = System.currentTimeMillis() val date = new Date(now) new DriverInfo(now, newDriverId(date), desc, date) } - def launchDriver(worker: WorkerInfo, driver: DriverInfo) { + private def launchDriver(worker: WorkerInfo, driver: DriverInfo) { logInfo("Launching driver " + driver.id + " on worker " + worker.id) worker.addDriver(driver) driver.worker = Some(worker) @@ -841,7 +845,10 @@ private[spark] class Master( driver.state = DriverState.RUNNING } - def removeDriver(driverId: String, finalState: DriverState, exception: Option[Exception]) { + private def removeDriver( + driverId: String, + finalState: DriverState, + exception: Option[Exception]) { drivers.find(d => d.id == driverId) match { case Some(driver) => logInfo(s"Removing driver: $driverId") @@ -862,7 +869,7 @@ private[spark] class Master( } } -private[spark] object Master extends Logging { +private[deploy] object Master extends Logging { val systemName = "sparkMaster" private val actorName = "Master" diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index e34bee7854292..435b9b12f83b8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -23,7 +23,7 @@ import org.apache.spark.util.{IntParam, Utils} /** * Command-line parser for the master. */ -private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { +private[master] class MasterArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 7077 var webUiPort = 8080 @@ -49,7 +49,7 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { webUiPort = conf.get("spark.master.ui.port").toInt } - def parse(args: List[String]): Unit = args match { + private def parse(args: List[String]): Unit = args match { case ("--ip" | "-i") :: value :: tail => Utils.checkHost(value, "ip no longer supported, please use hostname " + value) host = value @@ -84,7 +84,7 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { /** * Print usage and exit JVM with the given exit code. */ - def printUsageAndExit(exitCode: Int) { + private def printUsageAndExit(exitCode: Int) { System.err.println( "Usage: Master [options]\n" + "\n" + diff --git a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala index 2e0e1e7036ac8..da5060778edeb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala @@ -87,7 +87,7 @@ trait PersistenceEngine { def close() {} } -private[spark] class BlackHolePersistenceEngine extends PersistenceEngine { +private[master] class BlackHolePersistenceEngine extends PersistenceEngine { override def persist(name: String, obj: Object): Unit = {} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala index 1096eb0368357..1583bf1f60032 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala @@ -49,7 +49,7 @@ abstract class StandaloneRecoveryModeFactory(conf: SparkConf, serializer: Serial * LeaderAgent in this case is a no-op. Since leader is forever leader as the actual * recovery is made by restoring from filesystem. */ -private[spark] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: Serialization) +private[master] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: Serialization) extends StandaloneRecoveryModeFactory(conf, serializer) with Logging { val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") @@ -61,7 +61,7 @@ private[spark] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: def createLeaderElectionAgent(master: LeaderElectable) = new MonarchyLeaderAgent(master) } -private[spark] class ZooKeeperRecoveryModeFactory(conf: SparkConf, serializer: Serialization) +private[master] class ZooKeeperRecoveryModeFactory(conf: SparkConf, serializer: Serialization) extends StandaloneRecoveryModeFactory(conf, serializer) { def createPersistenceEngine() = new ZooKeeperPersistenceEngine(conf, serializer) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala index 256a5a7c28e47..aa0f02fa625cc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object RecoveryState extends Enumeration { +private[deploy] object RecoveryState extends Enumeration { type MasterState = Value val STANDBY, ALIVE, RECOVERING, COMPLETING_RECOVERY = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala index 4781a80d470e1..5b22481ea8c5f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala @@ -25,12 +25,12 @@ import org.apache.zookeeper.KeeperException import org.apache.spark.{Logging, SparkConf} -object SparkCuratorUtil extends Logging { +private[deploy] object SparkCuratorUtil extends Logging { - val ZK_CONNECTION_TIMEOUT_MILLIS = 15000 - val ZK_SESSION_TIMEOUT_MILLIS = 60000 - val RETRY_WAIT_MILLIS = 5000 - val MAX_RECONNECT_ATTEMPTS = 3 + private val ZK_CONNECTION_TIMEOUT_MILLIS = 15000 + private val ZK_SESSION_TIMEOUT_MILLIS = 60000 + private val RETRY_WAIT_MILLIS = 5000 + private val MAX_RECONNECT_ATTEMPTS = 3 def newClient(conf: SparkConf): CuratorFramework = { val ZK_URL = conf.get("spark.deploy.zookeeper.url") diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala index 0b36ef60051fc..b60baaadfb4bc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object WorkerState extends Enumeration { +private[master] object WorkerState extends Enumeration { type WorkerState = Value val ALIVE, DEAD, DECOMMISSIONED, UNKNOWN = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 8eaa0ad948519..4823fd7cac0cb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -24,7 +24,7 @@ import org.apache.spark.deploy.master.MasterMessages._ import org.apache.curator.framework.CuratorFramework import org.apache.curator.framework.recipes.leader.{LeaderLatchListener, LeaderLatch} -private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: LeaderElectable, +private[master] class ZooKeeperLeaderElectionAgent(val masterActor: LeaderElectable, conf: SparkConf) extends LeaderLatchListener with LeaderElectionAgent with Logging { val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" @@ -35,7 +35,7 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: LeaderElectab start() - def start() { + private def start() { logInfo("Starting ZooKeeper LeaderElection agent") zk = SparkCuratorUtil.newClient(conf) leaderLatch = new LeaderLatch(zk, WORKING_DIR) @@ -72,7 +72,7 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: LeaderElectab } } - def updateLeadershipStatus(isLeader: Boolean) { + private def updateLeadershipStatus(isLeader: Boolean) { if (isLeader && status == LeadershipStatus.NOT_LEADER) { status = LeadershipStatus.LEADER masterActor.electedLeader() diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index e11ac031fb9c6..1ac6677ad2b6d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -28,12 +28,12 @@ import org.apache.zookeeper.CreateMode import org.apache.spark.{Logging, SparkConf} -private[spark] class ZooKeeperPersistenceEngine(conf: SparkConf, val serialization: Serialization) +private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serialization: Serialization) extends PersistenceEngine - with Logging -{ - val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" - val zk: CuratorFramework = SparkCuratorUtil.newClient(conf) + with Logging { + + private val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" + private val zk: CuratorFramework = SparkCuratorUtil.newClient(conf) SparkCuratorUtil.mkdir(zk, WORKING_DIR) @@ -61,7 +61,7 @@ private[spark] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializati zk.create().withMode(CreateMode.PERSISTENT).forPath(path, serialized) } - def deserializeFromFile[T](filename: String)(implicit m: ClassTag[T]): Option[T] = { + private def deserializeFromFile[T](filename: String)(implicit m: ClassTag[T]): Option[T] = { val fileData = zk.getData().forPath(WORKING_DIR + "/" + filename) val clazz = m.runtimeClass.asInstanceOf[Class[T]] val serializer = serialization.serializerFor(clazz) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 76fc40e17d9a8..761aa8f7b1ef6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -32,7 +32,7 @@ import org.apache.spark.deploy.master.ExecutorDesc import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils -private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") { +private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") { private val master = parent.masterActorRef private val timeout = parent.timeout diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala index d8daff3e7fb9c..e021f1eef794f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala @@ -24,7 +24,7 @@ import scala.xml.Node import org.apache.spark.ui.{UIUtils, WebUIPage} -private[spark] class HistoryNotFoundPage(parent: MasterWebUI) +private[ui] class HistoryNotFoundPage(parent: MasterWebUI) extends WebUIPage("history/not-found") { /** diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index c086cadca2c7d..dee2e4a447c6e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.master._ import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { +private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { private val master = parent.masterActorRef private val timeout = parent.timeout diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 170f90a00ad2a..1b670418ab1ff 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -26,7 +26,7 @@ import org.apache.spark.util.AkkaUtils /** * Web UI server for the standalone master. */ -private[spark] +private[master] class MasterWebUI(val master: Master, requestedPort: Int) extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging { @@ -62,6 +62,6 @@ class MasterWebUI(val master: Master, requestedPort: Int) } } -private[spark] object MasterWebUI { - val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR +private[master] object MasterWebUI { + private val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala index c4be1f19e8e9f..420442f7564cc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala @@ -52,7 +52,7 @@ import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} * implementation of this client can use that information to retry using the version specified * by the server. */ -private[spark] class StandaloneRestClient extends Logging { +private[deploy] class StandaloneRestClient extends Logging { import StandaloneRestClient._ /** @@ -61,7 +61,7 @@ private[spark] class StandaloneRestClient extends Logging { * If the submission was successful, poll the status of the submission and report * it to the user. Otherwise, report the error message provided by the server. */ - def createSubmission( + private[rest] def createSubmission( master: String, request: CreateSubmissionRequest): SubmitRestProtocolResponse = { logInfo(s"Submitting a request to launch an application in $master.") @@ -106,7 +106,7 @@ private[spark] class StandaloneRestClient extends Logging { } /** Construct a message that captures the specified parameters for submitting an application. */ - def constructSubmitRequest( + private[rest] def constructSubmitRequest( appResource: String, mainClass: String, appArgs: Array[String], @@ -291,16 +291,16 @@ private[spark] class StandaloneRestClient extends Logging { } } -private[spark] object StandaloneRestClient { - val REPORT_DRIVER_STATUS_INTERVAL = 1000 - val REPORT_DRIVER_STATUS_MAX_TRIES = 10 +private[rest] object StandaloneRestClient { + private val REPORT_DRIVER_STATUS_INTERVAL = 1000 + private val REPORT_DRIVER_STATUS_MAX_TRIES = 10 val PROTOCOL_VERSION = "v1" /** * Submit an application, assuming Spark parameters are specified through the given config. * This is abstracted to its own method for testing purposes. */ - private[rest] def run( + def run( appResource: String, mainClass: String, appArgs: Array[String], diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index f9e0478e4f874..4f19af59f409f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -58,7 +58,7 @@ import org.apache.spark.deploy.ClientArguments._ * @param masterUrl the URL of the Master new drivers will attempt to connect to * @param masterConf the conf used by the Master */ -private[spark] class StandaloneRestServer( +private[deploy] class StandaloneRestServer( host: String, requestedPort: Int, masterActor: ActorRef, diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.scala index d7a0bdbe10778..b97921ec934a0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.scala @@ -20,17 +20,17 @@ package org.apache.spark.deploy.rest /** * An exception thrown in the REST application submission protocol. */ -private[spark] class SubmitRestProtocolException(message: String, cause: Throwable = null) +private[rest] class SubmitRestProtocolException(message: String, cause: Throwable = null) extends Exception(message, cause) /** * An exception thrown if a field is missing from a [[SubmitRestProtocolMessage]]. */ -private[spark] class SubmitRestMissingFieldException(message: String) +private[rest] class SubmitRestMissingFieldException(message: String) extends SubmitRestProtocolException(message) /** * An exception thrown if the REST client cannot reach the REST server. */ -private[spark] class SubmitRestConnectionException(message: String, cause: Throwable) +private[deploy] class SubmitRestConnectionException(message: String, cause: Throwable) extends SubmitRestProtocolException(message, cause) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala index 8f36635674a28..e6615a3174ce1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala @@ -39,7 +39,7 @@ import org.apache.spark.util.Utils @JsonInclude(Include.NON_NULL) @JsonAutoDetect(getterVisibility = Visibility.ANY, setterVisibility = Visibility.ANY) @JsonPropertyOrder(alphabetic = true) -private[spark] abstract class SubmitRestProtocolMessage { +private[rest] abstract class SubmitRestProtocolMessage { @JsonIgnore val messageType = Utils.getFormattedClassName(this) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala index 9e1fd8c40cabd..d80abdf15fb34 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala @@ -24,7 +24,7 @@ import org.apache.spark.util.Utils /** * An abstract request sent from the client in the REST application submission protocol. */ -private[spark] abstract class SubmitRestProtocolRequest extends SubmitRestProtocolMessage { +private[rest] abstract class SubmitRestProtocolRequest extends SubmitRestProtocolMessage { var clientSparkVersion: String = null protected override def doValidate(): Unit = { super.doValidate() @@ -35,7 +35,7 @@ private[spark] abstract class SubmitRestProtocolRequest extends SubmitRestProtoc /** * A request to launch a new application in the REST application submission protocol. */ -private[spark] class CreateSubmissionRequest extends SubmitRestProtocolRequest { +private[rest] class CreateSubmissionRequest extends SubmitRestProtocolRequest { var appResource: String = null var mainClass: String = null var appArgs: Array[String] = null diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala index 16dfe041d4bea..8fde8c142a4c1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala @@ -22,7 +22,7 @@ import java.lang.Boolean /** * An abstract response sent from the server in the REST application submission protocol. */ -private[spark] abstract class SubmitRestProtocolResponse extends SubmitRestProtocolMessage { +private[rest] abstract class SubmitRestProtocolResponse extends SubmitRestProtocolMessage { var serverSparkVersion: String = null var success: Boolean = null var unknownFields: Array[String] = null @@ -35,7 +35,7 @@ private[spark] abstract class SubmitRestProtocolResponse extends SubmitRestProto /** * A response to a [[CreateSubmissionRequest]] in the REST application submission protocol. */ -private[spark] class CreateSubmissionResponse extends SubmitRestProtocolResponse { +private[rest] class CreateSubmissionResponse extends SubmitRestProtocolResponse { var submissionId: String = null protected override def doValidate(): Unit = { super.doValidate() @@ -46,7 +46,7 @@ private[spark] class CreateSubmissionResponse extends SubmitRestProtocolResponse /** * A response to a kill request in the REST application submission protocol. */ -private[spark] class KillSubmissionResponse extends SubmitRestProtocolResponse { +private[rest] class KillSubmissionResponse extends SubmitRestProtocolResponse { var submissionId: String = null protected override def doValidate(): Unit = { super.doValidate() @@ -58,7 +58,7 @@ private[spark] class KillSubmissionResponse extends SubmitRestProtocolResponse { /** * A response to a status request in the REST application submission protocol. */ -private[spark] class SubmissionStatusResponse extends SubmitRestProtocolResponse { +private[rest] class SubmissionStatusResponse extends SubmitRestProtocolResponse { var submissionId: String = null var driverState: String = null var workerId: String = null @@ -74,7 +74,7 @@ private[spark] class SubmissionStatusResponse extends SubmitRestProtocolResponse /** * An error response message used in the REST application submission protocol. */ -private[spark] class ErrorResponse extends SubmitRestProtocolResponse { +private[rest] class ErrorResponse extends SubmitRestProtocolResponse { // The highest protocol version that the server knows about // This is set when the client specifies an unknown version var highestProtocolVersion: String = null diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 83f78cf47306c..0a1d60f58bc58 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.Utils /** ** Utilities for running commands with the spark classpath. */ -private[spark] +private[deploy] object CommandUtils extends Logging { /** diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index e16bccb24d2c4..27a9eabb1ede7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -37,8 +37,8 @@ import org.apache.spark.util.{Clock, SystemClock} * Manages the execution of one driver, including automatically restarting the driver on failure. * This is currently only used in standalone cluster deploy mode. */ -private[spark] class DriverRunner( - val conf: SparkConf, +private[deploy] class DriverRunner( + conf: SparkConf, val driverId: String, val workDir: File, val sparkHome: File, @@ -47,24 +47,24 @@ private[spark] class DriverRunner( val workerUrl: String) extends Logging { - @volatile var process: Option[Process] = None - @volatile var killed = false + @volatile private var process: Option[Process] = None + @volatile private var killed = false // Populated once finished - var finalState: Option[DriverState] = None - var finalException: Option[Exception] = None - var finalExitCode: Option[Int] = None + private[worker] var finalState: Option[DriverState] = None + private[worker] var finalException: Option[Exception] = None + private var finalExitCode: Option[Int] = None // Decoupled for testing - private[deploy] def setClock(_clock: Clock) = clock = _clock - private[deploy] def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper + def setClock(_clock: Clock) = clock = _clock + def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper private var clock: Clock = new SystemClock() private var sleeper = new Sleeper { def sleep(seconds: Int): Unit = (0 until seconds).takeWhile(f => {Thread.sleep(1000); !killed}) } /** Starts a thread to run and manage the driver. */ - def start() = { + private[worker] def start() = { new Thread("DriverRunner for " + driverId) { override def run() { try { @@ -106,7 +106,7 @@ private[spark] class DriverRunner( } /** Terminate this driver (or prevent it from ever starting if not yet started) */ - def kill() { + private[worker] def kill() { synchronized { process.foreach(p => p.destroy()) killed = true @@ -169,7 +169,7 @@ private[spark] class DriverRunner( runCommandWithRetry(ProcessBuilderLike(builder), initialize, supervise) } - private[deploy] def runCommandWithRetry(command: ProcessBuilderLike, initialize: Process => Unit, + def runCommandWithRetry(command: ProcessBuilderLike, initialize: Process => Unit, supervise: Boolean) { // Time to wait between submission retries. var waitSeconds = 1 diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 023f3c6269062..83e24a7a1f80c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.logging.FileAppender * Manages the execution of one executor process. * This is currently only used in standalone mode. */ -private[spark] class ExecutorRunner( +private[deploy] class ExecutorRunner( val appId: String, val execId: Int, val appDesc: ApplicationDescription, @@ -48,22 +48,22 @@ private[spark] class ExecutorRunner( val sparkHome: File, val executorDir: File, val workerUrl: String, - val conf: SparkConf, + conf: SparkConf, val appLocalDirs: Seq[String], var state: ExecutorState.Value) extends Logging { - val fullId = appId + "/" + execId - var workerThread: Thread = null - var process: Process = null - var stdoutAppender: FileAppender = null - var stderrAppender: FileAppender = null + private val fullId = appId + "/" + execId + private var workerThread: Thread = null + private var process: Process = null + private var stdoutAppender: FileAppender = null + private var stderrAppender: FileAppender = null // NOTE: This is now redundant with the automated shut-down enforced by the Executor. It might // make sense to remove this in the future. - var shutdownHook: Thread = null + private var shutdownHook: Thread = null - def start() { + private[worker] def start() { workerThread = new Thread("ExecutorRunner for " + fullId) { override def run() { fetchAndRunExecutor() } } @@ -99,7 +99,7 @@ private[spark] class ExecutorRunner( } /** Stop this executor runner, including killing the process it launched */ - def kill() { + private[worker] def kill() { if (workerThread != null) { // the workerThread will kill the child process when interrupted workerThread.interrupt() @@ -114,7 +114,7 @@ private[spark] class ExecutorRunner( } /** Replace variables such as {{EXECUTOR_ID}} and {{CORES}} in a command argument passed to us */ - def substituteVariables(argument: String): String = argument match { + private[worker] def substituteVariables(argument: String): String = argument match { case "{{WORKER_URL}}" => workerUrl case "{{EXECUTOR_ID}}" => execId.toString case "{{HOSTNAME}}" => host @@ -126,7 +126,7 @@ private[spark] class ExecutorRunner( /** * Download and run the executor described in our ApplicationDescription */ - def fetchAndRunExecutor() { + private def fetchAndRunExecutor() { try { // Launch the process val builder = CommandUtils.buildProcessBuilder(appDesc.command, memory, diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index f2e7418f4bf15..c1b0a295f9f74 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -42,7 +42,7 @@ import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} /** * @param masterAkkaUrls Each url should be a valid akka url. */ -private[spark] class Worker( +private[worker] class Worker( host: String, port: Int, webUiPort: Int, @@ -60,85 +60,90 @@ private[spark] class Worker( Utils.checkHost(host, "Expected hostname") assert (port > 0) - def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs + // For worker and executor IDs + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // Send a heartbeat every (heartbeat timeout) / 4 milliseconds - val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 + private val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 // Model retries to connect to the master, after Hadoop's model. // The first six attempts to reconnect are in shorter intervals (between 5 and 15 seconds) // Afterwards, the next 10 attempts are between 30 and 90 seconds. // A bit of randomness is introduced so that not all of the workers attempt to reconnect at // the same time. - val INITIAL_REGISTRATION_RETRIES = 6 - val TOTAL_REGISTRATION_RETRIES = INITIAL_REGISTRATION_RETRIES + 10 - val FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND = 0.500 - val REGISTRATION_RETRY_FUZZ_MULTIPLIER = { + private val INITIAL_REGISTRATION_RETRIES = 6 + private val TOTAL_REGISTRATION_RETRIES = INITIAL_REGISTRATION_RETRIES + 10 + private val FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND = 0.500 + private val REGISTRATION_RETRY_FUZZ_MULTIPLIER = { val randomNumberGenerator = new Random(UUID.randomUUID.getMostSignificantBits) randomNumberGenerator.nextDouble + FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND } - val INITIAL_REGISTRATION_RETRY_INTERVAL = (math.round(10 * + private val INITIAL_REGISTRATION_RETRY_INTERVAL = (math.round(10 * REGISTRATION_RETRY_FUZZ_MULTIPLIER)).seconds - val PROLONGED_REGISTRATION_RETRY_INTERVAL = (math.round(60 + private val PROLONGED_REGISTRATION_RETRY_INTERVAL = (math.round(60 * REGISTRATION_RETRY_FUZZ_MULTIPLIER)).seconds - val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) + private val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) // How often worker will clean up old app folders - val CLEANUP_INTERVAL_MILLIS = conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000 + private val CLEANUP_INTERVAL_MILLIS = + conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000 // TTL for app folders/data; after TTL expires it will be cleaned up - val APP_DATA_RETENTION_SECS = conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) - - val testing: Boolean = sys.props.contains("spark.testing") - var master: ActorSelection = null - var masterAddress: Address = null - var activeMasterUrl: String = "" - var activeMasterWebUiUrl : String = "" - val akkaUrl = AkkaUtils.address( + private val APP_DATA_RETENTION_SECS = + conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) + + private val testing: Boolean = sys.props.contains("spark.testing") + private var master: ActorSelection = null + private var masterAddress: Address = null + private var activeMasterUrl: String = "" + private[worker] var activeMasterWebUiUrl : String = "" + private val akkaUrl = AkkaUtils.address( AkkaUtils.protocol(context.system), actorSystemName, host, port, actorName) - @volatile var registered = false - @volatile var connected = false - val workerId = generateWorkerId() - val sparkHome = + @volatile private var registered = false + @volatile private var connected = false + private val workerId = generateWorkerId() + private val sparkHome = if (testing) { assert(sys.props.contains("spark.test.home"), "spark.test.home is not set!") new File(sys.props("spark.test.home")) } else { new File(sys.env.get("SPARK_HOME").getOrElse(".")) } + var workDir: File = null - val executors = new HashMap[String, ExecutorRunner] val finishedExecutors = new HashMap[String, ExecutorRunner] val drivers = new HashMap[String, DriverRunner] + val executors = new HashMap[String, ExecutorRunner] val finishedDrivers = new HashMap[String, DriverRunner] val appDirectories = new HashMap[String, Seq[String]] val finishedApps = new HashSet[String] // The shuffle service is not actually started unless configured. - val shuffleService = new StandaloneWorkerShuffleService(conf, securityMgr) + private val shuffleService = new StandaloneWorkerShuffleService(conf, securityMgr) - val publicAddress = { + private val publicAddress = { val envVar = conf.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host } - var webUi: WorkerWebUI = null + private var webUi: WorkerWebUI = null - var coresUsed = 0 - var memoryUsed = 0 - var connectionAttemptCount = 0 + private var connectionAttemptCount = 0 - val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) - val workerSource = new WorkerSource(this) + private val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) + private val workerSource = new WorkerSource(this) + + private var registrationRetryTimer: Option[Cancellable] = None - var registrationRetryTimer: Option[Cancellable] = None + var coresUsed = 0 + var memoryUsed = 0 def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed - def createWorkDir() { + private def createWorkDir() { workDir = Option(workDirPath).map(new File(_)).getOrElse(new File(sparkHome, "work")) try { // This sporadically fails - not sure why ... !workDir.exists() && !workDir.mkdirs() @@ -175,7 +180,7 @@ private[spark] class Worker( metricsSystem.getServletHandlers.foreach(webUi.attachHandler) } - def changeMaster(url: String, uiUrl: String) { + private def changeMaster(url: String, uiUrl: String) { // activeMasterUrl it's a valid Spark url since we receive it from master. activeMasterUrl = url activeMasterWebUiUrl = uiUrl @@ -252,7 +257,7 @@ private[spark] class Worker( } } - def registerWithMaster() { + private def registerWithMaster() { // DisassociatedEvent may be triggered multiple times, so don't attempt registration // if there are outstanding registration attempts scheduled. registrationRetryTimer match { @@ -506,7 +511,7 @@ private[spark] class Worker( } } - def generateWorkerId(): String = { + private def generateWorkerId(): String = { "worker-%s-%s-%d".format(createDateFormat.format(new Date), host, port) } @@ -521,7 +526,7 @@ private[spark] class Worker( } } -private[spark] object Worker extends Logging { +private[deploy] object Worker extends Logging { def main(argStrings: Array[String]) { SignalLogger.register(log) val conf = new SparkConf @@ -554,7 +559,7 @@ private[spark] object Worker extends Logging { (actorSystem, boundPort) } - private[spark] def isUseLocalNodeSSLConfig(cmd: Command): Boolean = { + def isUseLocalNodeSSLConfig(cmd: Command): Boolean = { val pattern = """\-Dspark\.ssl\.useNodeLocalConf\=(.+)""".r val result = cmd.javaOpts.collectFirst { case pattern(_result) => _result.toBoolean @@ -562,7 +567,7 @@ private[spark] object Worker extends Logging { result.getOrElse(false) } - private[spark] def maybeUpdateSSLSettings(cmd: Command, conf: SparkConf): Command = { + def maybeUpdateSSLSettings(cmd: Command, conf: SparkConf): Command = { val prefix = "spark.ssl." val useNLC = "spark.ssl.useNodeLocalConf" if (isUseLocalNodeSSLConfig(cmd)) { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 019cd70f2a229..88f9d880ac209 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -25,7 +25,7 @@ import org.apache.spark.SparkConf /** * Command-line parser for the worker. */ -private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) { +private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 0 var webUiPort = 8081 @@ -63,7 +63,7 @@ private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) { checkWorkerMemory() - def parse(args: List[String]): Unit = args match { + private def parse(args: List[String]): Unit = args match { case ("--ip" | "-i") :: value :: tail => Utils.checkHost(value, "ip no longer supported, please use hostname " + value) host = value diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala index df1e01b23b932..b36023bc40c3d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala @@ -21,7 +21,7 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.metrics.source.Source -private[spark] class WorkerSource(val worker: Worker) extends Source { +private[worker] class WorkerSource(val worker: Worker) extends Source { override val sourceName = "worker" override val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index 63a8ac817b618..09d866fb0cd90 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -48,7 +48,7 @@ private[spark] class WorkerWatcher(workerUrl: String) private val expectedHostPort = AddressFromURIString(workerUrl).hostPort private def isWorker(address: Address) = address.hostPort == expectedHostPort - def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1) + private def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1) override def receiveWithLogging = { case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index ecb358c399819..88170d4df3053 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -26,7 +26,7 @@ import org.apache.spark.util.Utils import org.apache.spark.Logging import org.apache.spark.util.logging.RollingFileAppender -private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with Logging { +private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with Logging { private val worker = parent.worker private val workDir = parent.workDir diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala index 720f13bfa829b..9f9f27d71e1ae 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala @@ -31,10 +31,9 @@ import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { - val workerActor = parent.worker.self - val worker = parent.worker - val timeout = parent.timeout +private[ui] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { + private val workerActor = parent.worker.self + private val timeout = parent.timeout override def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 7ac81a2d87efd..de6423beb543e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -30,7 +30,7 @@ import org.apache.spark.util.AkkaUtils /** * Web UI server for the standalone worker. */ -private[spark] +private[worker] class WorkerWebUI( val worker: Worker, val workDir: File, @@ -38,7 +38,7 @@ class WorkerWebUI( extends WebUI(worker.securityMgr, requestedPort, worker.conf, name = "WorkerUI") with Logging { - val timeout = AkkaUtils.askTimeout(worker.conf) + private[ui] val timeout = AkkaUtils.askTimeout(worker.conf) initialize() @@ -53,6 +53,6 @@ class WorkerWebUI( } } -private[spark] object WorkerWebUI { +private[ui] object WorkerWebUI { val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR } diff --git a/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala b/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala index f7604a321f007..f47d7ef511da1 100644 --- a/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala +++ b/core/src/main/scala/org/apache/spark/executor/CommitDeniedException.scala @@ -22,14 +22,12 @@ import org.apache.spark.{TaskCommitDenied, TaskEndReason} /** * Exception thrown when a task attempts to commit output to HDFS but is denied by the driver. */ -class CommitDeniedException( +private[spark] class CommitDeniedException( msg: String, jobID: Int, splitID: Int, attemptID: Int) extends Exception(msg) { - def toTaskEndReason: TaskEndReason = new TaskCommitDenied(jobID, splitID, attemptID) - + def toTaskEndReason: TaskEndReason = TaskCommitDenied(jobID, splitID, attemptID) } - diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index a897e532184ac..bf3135ef081c1 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -21,7 +21,7 @@ import java.io.File import java.lang.management.ManagementFactory import java.net.URL import java.nio.ByteBuffer -import java.util.concurrent._ +import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -31,15 +31,17 @@ import akka.actor.Props import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} -import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, - SparkUncaughtExceptionHandler, AkkaUtils, Utils} +import org.apache.spark.util._ /** - * Spark executor used with Mesos, YARN, and the standalone scheduler. - * In coarse-grained mode, an existing actor system is provided. + * Spark executor, backed by a threadpool to run tasks. + * + * This can be used with Mesos, YARN, and the standalone scheduler. + * An internal RPC interface (at the moment Akka) is used for communication with the driver, + * except in the case of Mesos fine-grained mode. */ private[spark] class Executor( executorId: String, @@ -47,8 +49,8 @@ private[spark] class Executor( env: SparkEnv, userClassPath: Seq[URL] = Nil, isLocal: Boolean = false) - extends Logging -{ + extends Logging { + logInfo(s"Starting executor ID $executorId on host $executorHostname") // Application dependencies (added through SparkContext) that we've fetched so far on this node. @@ -78,9 +80,8 @@ private[spark] class Executor( } // Start worker thread pool - val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker") - - val executorSource = new ExecutorSource(this, executorId) + private val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker") + private val executorSource = new ExecutorSource(threadPool, executorId) if (!isLocal) { env.metricsSystem.registerSource(executorSource) @@ -103,7 +104,7 @@ private[spark] class Executor( private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader) // Set the classloader for serializer - env.serializer.setDefaultClassLoader(urlClassLoader) + env.serializer.setDefaultClassLoader(replClassLoader) // Akka's message frame size. If task result is bigger than this, we use the block manager // to send the result back. @@ -122,21 +123,21 @@ private[spark] class Executor( taskId: Long, attemptNumber: Int, taskName: String, - serializedTask: ByteBuffer) { + serializedTask: ByteBuffer): Unit = { val tr = new TaskRunner(context, taskId = taskId, attemptNumber = attemptNumber, taskName, serializedTask) runningTasks.put(taskId, tr) threadPool.execute(tr) } - def killTask(taskId: Long, interruptThread: Boolean) { + def killTask(taskId: Long, interruptThread: Boolean): Unit = { val tr = runningTasks.get(taskId) if (tr != null) { tr.kill(interruptThread) } } - def stop() { + def stop(): Unit = { env.metricsSystem.report() env.actorSystem.stop(executorActor) isStopped = true @@ -146,7 +147,10 @@ private[spark] class Executor( } } - private def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum + /** Returns the total amount of time this JVM process has spent in garbage collection. */ + private def computeTotalGcTime(): Long = { + ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum + } class TaskRunner( execBackend: ExecutorBackend, @@ -156,12 +160,19 @@ private[spark] class Executor( serializedTask: ByteBuffer) extends Runnable { + /** Whether this task has been killed. */ @volatile private var killed = false - @volatile var task: Task[Any] = _ - @volatile var attemptedTask: Option[Task[Any]] = None + + /** How much the JVM process has spent in GC when the task starts to run. */ @volatile var startGCTime: Long = _ - def kill(interruptThread: Boolean) { + /** + * The task to run. This will be set in run() by deserializing the task binary coming + * from the driver. Once it is set, it will never be changed. + */ + @volatile var task: Task[Any] = _ + + def kill(interruptThread: Boolean): Unit = { logInfo(s"Executor is trying to kill $taskName (TID $taskId)") killed = true if (task != null) { @@ -169,14 +180,14 @@ private[spark] class Executor( } } - override def run() { + override def run(): Unit = { val deserializeStartTime = System.currentTimeMillis() Thread.currentThread.setContextClassLoader(replClassLoader) val ser = env.closureSerializer.newInstance() logInfo(s"Running $taskName (TID $taskId)") execBackend.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER) var taskStart: Long = 0 - startGCTime = gcTime + startGCTime = computeTotalGcTime() try { val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask) @@ -193,7 +204,6 @@ private[spark] class Executor( throw new TaskKilledException } - attemptedTask = Some(task) logDebug("Task " + taskId + "'s epoch is " + task.epoch) env.mapOutputTracker.updateEpoch(task.epoch) @@ -215,18 +225,17 @@ private[spark] class Executor( for (m <- task.metrics) { m.setExecutorDeserializeTime(taskStart - deserializeStartTime) m.setExecutorRunTime(taskFinish - taskStart) - m.setJvmGCTime(gcTime - startGCTime) + m.setJvmGCTime(computeTotalGcTime() - startGCTime) m.setResultSerializationTime(afterSerialization - beforeSerialization) } val accumUpdates = Accumulators.values - val directResult = new DirectTaskResult(valueBytes, accumUpdates, task.metrics.orNull) val serializedDirectResult = ser.serialize(directResult) val resultSize = serializedDirectResult.limit // directSend = sending directly back to the driver - val serializedResult = { + val serializedResult: ByteBuffer = { if (maxResultSize > 0 && resultSize > maxResultSize) { logWarning(s"Finished $taskName (TID $taskId). Result is larger than maxResultSize " + s"(${Utils.bytesToString(resultSize)} > ${Utils.bytesToString(maxResultSize)}), " + @@ -248,42 +257,40 @@ private[spark] class Executor( execBackend.statusUpdate(taskId, TaskState.FINISHED, serializedResult) } catch { - case ffe: FetchFailedException => { + case ffe: FetchFailedException => val reason = ffe.toTaskEndReason execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) - } - case _: TaskKilledException | _: InterruptedException if task.killed => { + case _: TaskKilledException | _: InterruptedException if task.killed => logInfo(s"Executor killed $taskName (TID $taskId)") execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) - } - case cDE: CommitDeniedException => { + case cDE: CommitDeniedException => val reason = cDE.toTaskEndReason execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) - } - case t: Throwable => { + case t: Throwable => // Attempt to exit cleanly by informing the driver of our failure. // If anything goes wrong (or this was a fatal exception), we will delegate to // the default uncaught exception handler, which will terminate the Executor. logError(s"Exception in $taskName (TID $taskId)", t) - val serviceTime = System.currentTimeMillis() - taskStart - val metrics = attemptedTask.flatMap(t => t.metrics) - for (m <- metrics) { - m.setExecutorRunTime(serviceTime) - m.setJvmGCTime(gcTime - startGCTime) + val metrics: Option[TaskMetrics] = Option(task).flatMap { task => + task.metrics.map { m => + m.setExecutorRunTime(System.currentTimeMillis() - taskStart) + m.setJvmGCTime(computeTotalGcTime() - startGCTime) + m + } } - val reason = new ExceptionFailure(t, metrics) - execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) + val taskEndReason = new ExceptionFailure(t, metrics) + execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(taskEndReason)) // Don't forcibly exit unless the exception was inherently fatal, to avoid // stopping other tasks unnecessarily. if (Utils.isFatalError(t)) { SparkUncaughtExceptionHandler.uncaughtException(t) } - } + } finally { // Release memory used by this thread for shuffles env.shuffleMemoryManager.releaseMemoryForThisThread() @@ -358,7 +365,7 @@ private[spark] class Executor( for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) // Fetch file with useCache mode, close cache for local mode. - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf, + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory()), conf, env.securityManager, hadoopConf, timestamp, useCache = !isLocal) currentFiles(name) = timestamp } @@ -370,12 +377,12 @@ private[spark] class Executor( if (currentTimeStamp < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) // Fetch file with useCache mode, close cache for local mode. - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf, + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory()), conf, env.securityManager, hadoopConf, timestamp, useCache = !isLocal) currentJars(name) = timestamp // Add it to our class loader - val url = new File(SparkFiles.getRootDirectory, localName).toURI.toURL - if (!urlClassLoader.getURLs.contains(url)) { + val url = new File(SparkFiles.getRootDirectory(), localName).toURI.toURL + if (!urlClassLoader.getURLs().contains(url)) { logInfo("Adding " + url + " to class loader") urlClassLoader.addURL(url) } @@ -384,61 +391,70 @@ private[spark] class Executor( } } - def startDriverHeartbeater() { - val interval = conf.getInt("spark.executor.heartbeatInterval", 10000) - val timeout = AkkaUtils.lookupTimeout(conf) - val retryAttempts = AkkaUtils.numRetries(conf) - val retryIntervalMs = AkkaUtils.retryWaitMs(conf) - val heartbeatReceiverRef = AkkaUtils.makeDriverRef("HeartbeatReceiver", conf, env.actorSystem) + private val timeout = AkkaUtils.lookupTimeout(conf) + private val retryAttempts = AkkaUtils.numRetries(conf) + private val retryIntervalMs = AkkaUtils.retryWaitMs(conf) + private val heartbeatReceiverRef = + AkkaUtils.makeDriverRef("HeartbeatReceiver", conf, env.actorSystem) + + /** Reports heartbeat and metrics for active tasks to the driver. */ + private def reportHeartBeat(): Unit = { + // list of (task id, metrics) to send back to the driver + val tasksMetrics = new ArrayBuffer[(Long, TaskMetrics)]() + val curGCTime = computeTotalGcTime() + + for (taskRunner <- runningTasks.values()) { + if (taskRunner.task != null) { + taskRunner.task.metrics.foreach { metrics => + metrics.updateShuffleReadMetrics() + metrics.updateInputMetrics() + metrics.setJvmGCTime(curGCTime - taskRunner.startGCTime) + + if (isLocal) { + // JobProgressListener will hold an reference of it during + // onExecutorMetricsUpdate(), then JobProgressListener can not see + // the changes of metrics any more, so make a deep copy of it + val copiedMetrics = Utils.deserialize[TaskMetrics](Utils.serialize(metrics)) + tasksMetrics += ((taskRunner.taskId, copiedMetrics)) + } else { + // It will be copied by serialization + tasksMetrics += ((taskRunner.taskId, metrics)) + } + } + } + } - val t = new Thread() { + val message = Heartbeat(executorId, tasksMetrics.toArray, env.blockManager.blockManagerId) + try { + val response = AkkaUtils.askWithReply[HeartbeatResponse](message, heartbeatReceiverRef, + retryAttempts, retryIntervalMs, timeout) + if (response.reregisterBlockManager) { + logWarning("Told to re-register on heartbeat") + env.blockManager.reregister() + } + } catch { + case NonFatal(e) => logWarning("Issue communicating with driver in heartbeater", e) + } + } + + /** + * Starts a thread to report heartbeat and partial metrics for active tasks to driver. + * This thread stops running when the executor is stopped. + */ + private def startDriverHeartbeater(): Unit = { + val interval = conf.getInt("spark.executor.heartbeatInterval", 10000) + val thread = new Thread() { override def run() { // Sleep a random interval so the heartbeats don't end up in sync Thread.sleep(interval + (math.random * interval).asInstanceOf[Int]) - while (!isStopped) { - val tasksMetrics = new ArrayBuffer[(Long, TaskMetrics)]() - val curGCTime = gcTime - - for (taskRunner <- runningTasks.values()) { - if (taskRunner.attemptedTask.nonEmpty) { - Option(taskRunner.task).flatMap(_.metrics).foreach { metrics => - metrics.updateShuffleReadMetrics() - metrics.updateInputMetrics() - metrics.setJvmGCTime(curGCTime - taskRunner.startGCTime) - - if (isLocal) { - // JobProgressListener will hold an reference of it during - // onExecutorMetricsUpdate(), then JobProgressListener can not see - // the changes of metrics any more, so make a deep copy of it - val copiedMetrics = Utils.deserialize[TaskMetrics](Utils.serialize(metrics)) - tasksMetrics += ((taskRunner.taskId, copiedMetrics)) - } else { - // It will be copied by serialization - tasksMetrics += ((taskRunner.taskId, metrics)) - } - } - } - } - - val message = Heartbeat(executorId, tasksMetrics.toArray, env.blockManager.blockManagerId) - try { - val response = AkkaUtils.askWithReply[HeartbeatResponse](message, heartbeatReceiverRef, - retryAttempts, retryIntervalMs, timeout) - if (response.reregisterBlockManager) { - logWarning("Told to re-register on heartbeat") - env.blockManager.reregister() - } - } catch { - case NonFatal(t) => logWarning("Issue communicating with driver in heartbeater", t) - } - + reportHeartBeat() Thread.sleep(interval) } } } - t.setDaemon(true) - t.setName("Driver Heartbeater") - t.start() + thread.setDaemon(true) + thread.setName("driver-heartbeater") + thread.start() } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index c4d73622c4727..293c512f8b70c 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -17,6 +17,8 @@ package org.apache.spark.executor +import java.util.concurrent.ThreadPoolExecutor + import scala.collection.JavaConversions._ import com.codahale.metrics.{Gauge, MetricRegistry} @@ -24,9 +26,11 @@ import org.apache.hadoop.fs.FileSystem import org.apache.spark.metrics.source.Source -private[spark] class ExecutorSource(val executor: Executor, executorId: String) extends Source { +private[spark] +class ExecutorSource(threadPool: ThreadPoolExecutor, executorId: String) extends Source { + private def fileStats(scheme: String) : Option[FileSystem.Statistics] = - FileSystem.getAllStatistics().filter(s => s.getScheme.equals(scheme)).headOption + FileSystem.getAllStatistics().find(s => s.getScheme.equals(scheme)) private def registerFileSystemStat[T]( scheme: String, name: String, f: FileSystem.Statistics => T, defaultValue: T) = { @@ -41,23 +45,23 @@ private[spark] class ExecutorSource(val executor: Executor, executorId: String) // Gauge for executor thread pool's actively executing task counts metricRegistry.register(MetricRegistry.name("threadpool", "activeTasks"), new Gauge[Int] { - override def getValue: Int = executor.threadPool.getActiveCount() + override def getValue: Int = threadPool.getActiveCount() }) // Gauge for executor thread pool's approximate total number of tasks that have been completed metricRegistry.register(MetricRegistry.name("threadpool", "completeTasks"), new Gauge[Long] { - override def getValue: Long = executor.threadPool.getCompletedTaskCount() + override def getValue: Long = threadPool.getCompletedTaskCount() }) // Gauge for executor thread pool's current number of threads metricRegistry.register(MetricRegistry.name("threadpool", "currentPool_size"), new Gauge[Int] { - override def getValue: Int = executor.threadPool.getPoolSize() + override def getValue: Int = threadPool.getPoolSize() }) // Gauge got executor thread pool's largest number of threads that have ever simultaneously // been in th pool metricRegistry.register(MetricRegistry.name("threadpool", "maxPool_size"), new Gauge[Int] { - override def getValue: Int = executor.threadPool.getMaximumPoolSize() + override def getValue: Int = threadPool.getMaximumPoolSize() }) // Gauge for file system stats of this executor diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index cf0433010aa03..a139780d967e9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -960,7 +960,7 @@ abstract class RDD[T: ClassTag]( */ def aggregate[U: ClassTag](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = { // Clone the zero value since we will also be serializing it as part of tasks - var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance()) + var jobResult = Utils.clone(zeroValue, sc.env.serializer.newInstance()) val cleanSeqOp = sc.clean(seqOp) val cleanCombOp = sc.clean(combOp) val aggregatePartition = (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index bc84e2351ad74..8feac6cb6b7a1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -26,7 +26,6 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map, Stack} import scala.concurrent.Await import scala.concurrent.duration._ import scala.language.postfixOps -import scala.reflect.ClassTag import scala.util.control.NonFatal import akka.pattern.ask @@ -105,7 +104,7 @@ class DAGScheduler( * * All accesses to this map should be guarded by synchronizing on it (see SPARK-4454). */ - private val cacheLocs = new HashMap[Int, Array[Seq[TaskLocation]]] + private val cacheLocs = new HashMap[Int, Seq[Seq[TaskLocation]]] // For tracking failed nodes, we use the MapOutputTracker's epoch number, which is sent with // every task. When we detect a node failing, we note the current epoch number and failed @@ -189,14 +188,15 @@ class DAGScheduler( eventProcessLoop.post(TaskSetFailed(taskSet, reason)) } - private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = cacheLocs.synchronized { + private[scheduler] + def getCacheLocs(rdd: RDD[_]): Seq[Seq[TaskLocation]] = cacheLocs.synchronized { // Note: this doesn't use `getOrElse()` because this method is called O(num tasks) times if (!cacheLocs.contains(rdd.id)) { val blockIds = rdd.partitions.indices.map(index => RDDBlockId(rdd.id, index)).toArray[BlockId] - val locs = BlockManager.blockIdsToBlockManagers(blockIds, env, blockManagerMaster) - cacheLocs(rdd.id) = blockIds.map { id => - locs.getOrElse(id, Nil).map(bm => TaskLocation(bm.host, bm.executorId)) + val locs: Seq[Seq[TaskLocation]] = blockManagerMaster.getLocations(blockIds).map { bms => + bms.map(bm => TaskLocation(bm.host, bm.executorId)) } + cacheLocs(rdd.id) = locs } cacheLocs(rdd.id) } @@ -497,7 +497,7 @@ class DAGScheduler( waiter } - def runJob[T, U: ClassTag]( + def runJob[T, U]( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, partitions: Seq[Int], @@ -1262,7 +1262,6 @@ class DAGScheduler( return true } val visitedRdds = new HashSet[RDD[_]] - val visitedStages = new HashSet[Stage] // We are manually maintaining a stack here to prevent StackOverflowError // caused by recursively visiting val waitingForVisit = new Stack[RDD[_]] @@ -1274,7 +1273,6 @@ class DAGScheduler( case shufDep: ShuffleDependency[_, _, _] => val mapStage = getShuffleMapStage(shufDep, stage.jobId) if (!mapStage.isAvailable) { - visitedStages += mapStage waitingForVisit.push(mapStage.rdd) } // Otherwise there's no need to follow the dependency back case narrowDep: NarrowDependency[_] => diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 2091a9fe8d0d3..34fa6d27c3a45 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -95,7 +95,7 @@ private[spark] class EventLoggingListener( * Creates the log file in the configured log directory. */ def start() { - if (!fileSystem.isDirectory(new Path(logBaseDir))) { + if (!fileSystem.getFileStatus(new Path(logBaseDir)).isDir) { throw new IllegalArgumentException(s"Log directory $logBaseDir does not exist.") } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 95273c716b3e2..86f357abb8723 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -21,6 +21,7 @@ import java.io.{InputStream, IOException} import scala.io.Source +import com.fasterxml.jackson.core.JsonParseException import org.json4s.jackson.JsonMethods._ import org.apache.spark.Logging @@ -40,15 +41,31 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { * * @param logData Stream containing event log data. * @param sourceName Filename (or other source identifier) from whence @logData is being read + * @param maybeTruncated Indicate whether log file might be truncated (some abnormal situations + * encountered, log file might not finished writing) or not */ - def replay(logData: InputStream, sourceName: String): Unit = { + def replay( + logData: InputStream, + sourceName: String, + maybeTruncated: Boolean = false): Unit = { var currentLine: String = null var lineNumber: Int = 1 try { val lines = Source.fromInputStream(logData).getLines() - lines.foreach { line => - currentLine = line - postToAll(JsonProtocol.sparkEventFromJson(parse(line))) + while (lines.hasNext) { + currentLine = lines.next() + try { + postToAll(JsonProtocol.sparkEventFromJson(parse(currentLine))) + } catch { + case jpe: JsonParseException => + // We can only ignore exception from last line of the file that might be truncated + if (!maybeTruncated || lines.hasNext) { + throw jpe + } else { + logWarning(s"Got JsonParseException from log file $sourceName" + + s" at line $lineNumber, the file might not have finished writing cleanly.") + } + } lineNumber += 1 } } catch { diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 847a4912eec13..4d9f940813b8e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -45,7 +45,7 @@ import org.apache.spark.util.Utils private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) extends Serializable { /** - * Called by Executor to run this task. + * Called by [[Executor]] to run this task. * * @param taskAttemptId an identifier for this task attempt that is unique within a SparkContext. * @param attemptNumber how many times this task has been attempted (0 for the first attempt) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 7a9cf1c2e7f30..f33fd4450b2a6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -145,7 +145,7 @@ private[spark] class TaskSchedulerImpl( import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, SPECULATION_INTERVAL milliseconds) { - Utils.tryOrExit { checkSpeculatableTasks() } + Utils.tryOrStopSparkContext(sc) { checkSpeculatableTasks() } } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 6f77fa32ce37b..87ebf31139ce9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -211,6 +211,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // This must be synchronized because variables mutated // in this block are read when requesting executors CoarseGrainedSchedulerBackend.this.synchronized { + addressToExecutorId -= executorInfo.executorAddress executorDataMap -= executorId executorsPendingToRemove -= executorId } @@ -371,6 +372,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste logWarning(s"Executor to kill $id does not exist!") } } + // Killing executors means effectively that we want less executors than before, so also update + // the target number of executors to avoid having the backend allocate new ones. + val newTotal = (numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size + - filteredExecutorIds.size) + doRequestTotalExecutors(newTotal) + executorsPendingToRemove ++= filteredExecutorIds doKillExecutors(filteredExecutorIds) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 90dfe14352a8e..fc92b9c35c3a3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -28,7 +28,7 @@ import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException} +import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException, TaskState} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.{Utils, AkkaUtils} @@ -262,20 +262,12 @@ private[spark] class CoarseMesosSchedulerBackend( .build() } - /** Check whether a Mesos task state represents a finished task */ - private def isFinished(state: MesosTaskState) = { - state == MesosTaskState.TASK_FINISHED || - state == MesosTaskState.TASK_FAILED || - state == MesosTaskState.TASK_KILLED || - state == MesosTaskState.TASK_LOST - } - override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { val taskId = status.getTaskId.getValue.toInt val state = status.getState logInfo("Mesos task " + taskId + " is now " + state) synchronized { - if (isFinished(state)) { + if (TaskState.isFinished(TaskState.fromMesos(state))) { val slaveId = taskIdToSlaveId(taskId) slaveIdsWithExecutors -= slaveId taskIdToSlaveId -= taskId diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index cfb6592e14aa8..df8f4306b88a8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -313,14 +313,6 @@ private[spark] class MesosSchedulerBackend( .build() } - /** Check whether a Mesos task state represents a finished task */ - def isFinished(state: MesosTaskState) = { - state == MesosTaskState.TASK_FINISHED || - state == MesosTaskState.TASK_FAILED || - state == MesosTaskState.TASK_KILLED || - state == MesosTaskState.TASK_LOST - } - override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { inClassLoader() { val tid = status.getTaskId.getValue.toLong @@ -330,7 +322,7 @@ private[spark] class MesosSchedulerBackend( // We lost the executor on this slave, so remember that it's gone removeExecutor(taskIdToSlaveId(tid), "Lost executor") } - if (isFinished(status.getState)) { + if (TaskState.isFinished(state)) { taskIdToSlaveId.remove(tid) } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 27496c5a289cb..fa2e617762f55 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -88,7 +88,10 @@ private[spark] class SortShuffleWriter[K, V, C]( } finally { // Clean up our sorter, which may have its own intermediate files if (sorter != null) { + val startTime = System.nanoTime() sorter.stop() + context.taskMetrics.shuffleWriteMetrics.foreach( + _.incShuffleWriteTime(System.nanoTime - startTime)) sorter = null } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index c8b7763f03fb7..80d66e59132da 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1245,10 +1245,10 @@ private[spark] object BlockManager extends Logging { } } - def blockIdsToBlockManagers( + def blockIdsToHosts( blockIds: Array[BlockId], env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[BlockManagerId]] = { + blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = { // blockManagerMaster != null is used in tests assert(env != null || blockManagerMaster != null) @@ -1258,24 +1258,10 @@ private[spark] object BlockManager extends Logging { blockManagerMaster.getLocations(blockIds) } - val blockManagers = new HashMap[BlockId, Seq[BlockManagerId]] + val blockManagers = new HashMap[BlockId, Seq[String]] for (i <- 0 until blockIds.length) { - blockManagers(blockIds(i)) = blockLocations(i) + blockManagers(blockIds(i)) = blockLocations(i).map(_.host) } blockManagers.toMap } - - def blockIdsToExecutorIds( - blockIds: Array[BlockId], - env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = { - blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.executorId)) - } - - def blockIdsToHosts( - blockIds: Array[BlockId], - env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = { - blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.host)) - } } diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index ec68837a1516c..ea548f23120d9 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -20,14 +20,15 @@ package org.apache.spark.ui import javax.servlet.http.HttpServletRequest import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap import scala.xml.Node import org.eclipse.jetty.servlet.ServletContextHandler import org.json4s.JsonAST.{JNothing, JValue} -import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.Utils +import org.apache.spark.{Logging, SecurityManager, SparkConf} /** * The top level component of the UI hierarchy that contains the server. @@ -45,6 +46,7 @@ private[spark] abstract class WebUI( protected val tabs = ArrayBuffer[WebUITab]() protected val handlers = ArrayBuffer[ServletContextHandler]() + protected val pageToHandlers = new HashMap[WebUIPage, ArrayBuffer[ServletContextHandler]] protected var serverInfo: Option[ServerInfo] = None protected val localHostName = Utils.localHostName() protected val publicHostName = Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName) @@ -60,14 +62,30 @@ private[spark] abstract class WebUI( tab.pages.foreach(attachPage) tabs += tab } + + def detachTab(tab: WebUITab) { + tab.pages.foreach(detachPage) + tabs -= tab + } + + def detachPage(page: WebUIPage) { + pageToHandlers.remove(page).foreach(_.foreach(detachHandler)) + } /** Attach a page to this UI. */ def attachPage(page: WebUIPage) { val pagePath = "/" + page.prefix - attachHandler(createServletHandler(pagePath, - (request: HttpServletRequest) => page.render(request), securityManager, basePath)) - attachHandler(createServletHandler(pagePath.stripSuffix("/") + "/json", - (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath)) + val renderHandler = createServletHandler(pagePath, + (request: HttpServletRequest) => page.render(request), securityManager, basePath) + val renderJsonHandler = createServletHandler(pagePath.stripSuffix("/") + "/json", + (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath) + attachHandler(renderHandler) + attachHandler(renderJsonHandler) + pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]()) + .append(renderHandler) + pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]()) + .append(renderJsonHandler) + } /** Attach a handler to this UI. */ diff --git a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala index 18c627e8c7a15..ce7887b76ff96 100644 --- a/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/AsynchronousListenerBus.scala @@ -21,6 +21,7 @@ import java.util.concurrent._ import java.util.concurrent.atomic.AtomicBoolean import com.google.common.annotations.VisibleForTesting +import org.apache.spark.SparkContext /** * Asynchronously passes events to registered listeners. @@ -38,6 +39,8 @@ private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: Stri self => + private var sparkContext: SparkContext = null + /* Cap the capacity of the event queue so we get an explicit error (rather than * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ private val EVENT_QUEUE_CAPACITY = 10000 @@ -57,7 +60,7 @@ private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: Stri private val listenerThread = new Thread(name) { setDaemon(true) - override def run(): Unit = Utils.logUncaughtExceptions { + override def run(): Unit = Utils.tryOrStopSparkContext(sparkContext) { while (true) { eventLock.acquire() self.synchronized { @@ -89,9 +92,12 @@ private[spark] abstract class AsynchronousListenerBus[L <: AnyRef, E](name: Stri * This first sends out all buffered events posted before this listener bus has started, then * listens for any additional events asynchronously while the listener bus is still running. * This should only be called once. + * + * @param sc Used to stop the SparkContext in case the listener thread dies. */ - def start() { + def start(sc: SparkContext) { if (started.compareAndSet(false, true)) { + sparkContext = sc listenerThread.start() } else { throw new IllegalStateException(s"$name already started!") diff --git a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala index d9c7103b2f3bf..1e0ba5c28754a 100644 --- a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala @@ -23,8 +23,6 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions._ -import org.apache.spark.util.ParentClassLoader - /** * URL class loader that exposes the `addURL` and `getURLs` methods in URLClassLoader. */ diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index bce3b3afe9aba..26ffbf9350388 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -18,18 +18,16 @@ package org.apache.spark.util import java.lang.management.ManagementFactory -import java.lang.reflect.{Array => JArray} -import java.lang.reflect.Field -import java.lang.reflect.Modifier -import java.util.IdentityHashMap -import java.util.Random +import java.lang.reflect.{Field, Modifier} +import java.util.{IdentityHashMap, Random} import java.util.concurrent.ConcurrentHashMap - import scala.collection.mutable.ArrayBuffer +import scala.runtime.ScalaRunTime import org.apache.spark.Logging import org.apache.spark.util.collection.OpenHashSet + /** * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in * memory-aware caches. @@ -184,9 +182,9 @@ private[spark] object SizeEstimator extends Logging { private val ARRAY_SIZE_FOR_SAMPLING = 200 private val ARRAY_SAMPLE_SIZE = 100 // should be lower than ARRAY_SIZE_FOR_SAMPLING - private def visitArray(array: AnyRef, cls: Class[_], state: SearchState) { - val length = JArray.getLength(array) - val elementClass = cls.getComponentType + private def visitArray(array: AnyRef, arrayClass: Class[_], state: SearchState) { + val length = ScalaRunTime.array_length(array) + val elementClass = arrayClass.getComponentType() // Arrays have object header and length field which is an integer var arrSize: Long = alignSize(objectSize + INT_SIZE) @@ -199,22 +197,26 @@ private[spark] object SizeEstimator extends Logging { state.size += arrSize if (length <= ARRAY_SIZE_FOR_SAMPLING) { - for (i <- 0 until length) { - state.enqueue(JArray.get(array, i)) + var arrayIndex = 0 + while (arrayIndex < length) { + state.enqueue(ScalaRunTime.array_apply(array, arrayIndex).asInstanceOf[AnyRef]) + arrayIndex += 1 } } else { // Estimate the size of a large array by sampling elements without replacement. var size = 0.0 val rand = new Random(42) val drawn = new OpenHashSet[Int](ARRAY_SAMPLE_SIZE) - for (i <- 0 until ARRAY_SAMPLE_SIZE) { + var numElementsDrawn = 0 + while (numElementsDrawn < ARRAY_SAMPLE_SIZE) { var index = 0 do { index = rand.nextInt(length) } while (drawn.contains(index)) drawn.add(index) - val elem = JArray.get(array, index) + val elem = ScalaRunTime.array_apply(array, index).asInstanceOf[AnyRef] size += SizeEstimator.estimate(elem, state.visited) + numElementsDrawn += 1 } state.size += ((length / (ARRAY_SAMPLE_SIZE * 1.0)) * size).toLong } 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 d3dc1d09cb7b4..91aa70870ab20 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -403,7 +403,8 @@ private[spark] object Utils extends Logging { useCache: Boolean) { val fileName = url.split("/").last val targetFile = new File(targetDir, fileName) - if (useCache) { + val fetchCacheEnabled = conf.getBoolean("spark.files.useFetchCache", defaultValue = true) + if (useCache && fetchCacheEnabled) { val cachedFileName = s"${url.hashCode}${timestamp}_cache" val lockFileName = s"${url.hashCode}${timestamp}_lock" val localDir = new File(getLocalDir(conf)) @@ -1145,6 +1146,8 @@ private[spark] object Utils extends Logging { /** * Execute a block of code that evaluates to Unit, forwarding any uncaught exceptions to the * default UncaughtExceptionHandler + * + * NOTE: This method is to be called by the spark-started JVM process. */ def tryOrExit(block: => Unit) { try { @@ -1155,6 +1158,32 @@ private[spark] object Utils extends Logging { } } + /** + * Execute a block of code that evaluates to Unit, stop SparkContext is there is any uncaught + * exception + * + * NOTE: This method is to be called by the driver-side components to avoid stopping the + * user-started JVM process completely; in contrast, tryOrExit is to be called in the + * spark-started JVM process . + */ + def tryOrStopSparkContext(sc: SparkContext)(block: => Unit) { + try { + block + } catch { + case e: ControlThrowable => throw e + case t: Throwable => + val currentThreadName = Thread.currentThread().getName + if (sc != null) { + logError(s"uncaught error in thread $currentThreadName, stopping SparkContext", t) + sc.stop() + } + if (!NonFatal(t)) { + logError(s"throw uncaught fatal error in thread $currentThreadName", t) + throw t + } + } + } + /** * Execute a block of code that evaluates to Unit, re-throwing any non-fatal uncaught * exceptions as IOException. This is used when implementing Externalizable and Serializable's diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 74e88c767ee07..d4b5bb519157c 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -24,11 +24,12 @@ import java.util.*; import java.util.concurrent.*; -import org.apache.spark.input.PortableDataStream; +import scala.collection.JavaConversions; import scala.Tuple2; import scala.Tuple3; import scala.Tuple4; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; @@ -51,8 +52,11 @@ import org.apache.spark.api.java.*; import org.apache.spark.api.java.function.*; import org.apache.spark.executor.TaskMetrics; +import org.apache.spark.input.PortableDataStream; import org.apache.spark.partial.BoundedDouble; import org.apache.spark.partial.PartialResult; +import org.apache.spark.rdd.RDD; +import org.apache.spark.serializer.KryoSerializer; import org.apache.spark.storage.StorageLevel; import org.apache.spark.util.StatCounter; @@ -267,6 +271,22 @@ public void call(String s) throws IOException { Assert.assertEquals(2, accum.value().intValue()); } + @Test + public void foreachPartition() { + final Accumulator accum = sc.accumulator(0); + JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); + rdd.foreachPartition(new VoidFunction>() { + @Override + public void call(Iterator iter) throws IOException { + while (iter.hasNext()) { + iter.next(); + accum.add(1); + } + } + }); + Assert.assertEquals(2, accum.value().intValue()); + } + @Test public void toLocalIterator() { List correct = Arrays.asList(1, 2, 3, 4); @@ -657,6 +677,13 @@ public Boolean call(Integer i) { }).isEmpty()); } + @Test + public void toArray() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3)); + List list = rdd.toArray(); + Assert.assertEquals(Arrays.asList(1, 2, 3), list); + } + @Test public void cartesian() { JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0)); @@ -703,8 +730,8 @@ public void javaDoubleRDDHistoGram() { Tuple2 results = rdd.histogram(2); double[] expected_buckets = {1.0, 2.5, 4.0}; long[] expected_counts = {2, 2}; - Assert.assertArrayEquals(expected_buckets, results._1, 0.1); - Assert.assertArrayEquals(expected_counts, results._2); + Assert.assertArrayEquals(expected_buckets, results._1(), 0.1); + Assert.assertArrayEquals(expected_counts, results._2()); // Test with provided buckets long[] histogram = rdd.histogram(expected_buckets); Assert.assertArrayEquals(expected_counts, histogram); @@ -714,6 +741,80 @@ public void javaDoubleRDDHistoGram() { sc.parallelizeDoubles(new ArrayList(0), 1).histogram(new double[]{0.0, 1.0})); } + private static class DoubleComparator implements Comparator, Serializable { + public int compare(Double o1, Double o2) { + return o1.compareTo(o2); + } + } + + @Test + public void max() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double max = rdd.max(new DoubleComparator()); + Assert.assertEquals(4.0, max, 0.001); + } + + @Test + public void min() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double max = rdd.min(new DoubleComparator()); + Assert.assertEquals(1.0, max, 0.001); + } + + @Test + public void takeOrdered() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + Assert.assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2, new DoubleComparator())); + Assert.assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2)); + } + + @Test + public void top() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + List top2 = rdd.top(2); + Assert.assertEquals(Arrays.asList(4, 3), top2); + } + + private static class AddInts implements Function2 { + @Override + public Integer call(Integer a, Integer b) { + return a + b; + } + } + + @Test + public void reduce() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + int sum = rdd.reduce(new AddInts()); + Assert.assertEquals(10, sum); + } + + @Test + public void reduceOnJavaDoubleRDD() { + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + double sum = rdd.reduce(new Function2() { + @Override + public Double call(Double v1, Double v2) throws Exception { + return v1 + v2; + } + }); + Assert.assertEquals(10.0, sum, 0.001); + } + + @Test + public void fold() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + int sum = rdd.fold(0, new AddInts()); + Assert.assertEquals(10, sum); + } + + @Test + public void aggregate() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + int sum = rdd.aggregate(0, new AddInts(), new AddInts()); + Assert.assertEquals(10, sum); + } + @Test public void map() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); @@ -830,6 +931,25 @@ public Iterable call(Iterator iter) { Assert.assertEquals("[3, 7]", partitionSums.collect().toString()); } + + @Test + public void mapPartitionsWithIndex() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); + JavaRDD partitionSums = rdd.mapPartitionsWithIndex( + new Function2, Iterator>() { + @Override + public Iterator call(Integer index, Iterator iter) throws Exception { + int sum = 0; + while (iter.hasNext()) { + sum += iter.next(); + } + return Collections.singletonList(sum).iterator(); + } + }, false); + Assert.assertEquals("[3, 7]", partitionSums.collect().toString()); + } + + @Test public void repartition() { // Shrinking number of partitions @@ -1308,6 +1428,49 @@ public void checkpointAndRestore() { Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), recovered.collect()); } + @Test + public void combineByKey() { + JavaRDD originalRDD = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6)); + Function keyFunction = new Function() { + @Override + public Integer call(Integer v1) throws Exception { + return v1 % 3; + } + }; + Function createCombinerFunction = new Function() { + @Override + public Integer call(Integer v1) throws Exception { + return v1; + } + }; + + Function2 mergeValueFunction = new Function2() { + @Override + public Integer call(Integer v1, Integer v2) throws Exception { + return v1 + v2; + } + }; + + JavaPairRDD combinedRDD = originalRDD.keyBy(keyFunction) + .combineByKey(createCombinerFunction, mergeValueFunction, mergeValueFunction); + Map results = combinedRDD.collectAsMap(); + ImmutableMap expected = ImmutableMap.of(0, 9, 1, 5, 2, 7); + Assert.assertEquals(expected, results); + + Partitioner defaultPartitioner = Partitioner.defaultPartitioner( + combinedRDD.rdd(), JavaConversions.asScalaBuffer(Lists.>newArrayList())); + combinedRDD = originalRDD.keyBy(keyFunction) + .combineByKey( + createCombinerFunction, + mergeValueFunction, + mergeValueFunction, + defaultPartitioner, + false, + new KryoSerializer(new SparkConf())); + results = combinedRDD.collectAsMap(); + Assert.assertEquals(expected, results); + } + @SuppressWarnings("unchecked") @Test public void mapOnPairRDD() { @@ -1516,6 +1679,19 @@ public void collectAsync() throws Exception { Assert.assertEquals(1, future.jobIds().size()); } + @Test + public void takeAsync() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction> future = rdd.takeAsync(1); + List result = future.get(); + Assert.assertEquals(1, result.size()); + Assert.assertEquals((Integer) 1, result.get(0)); + Assert.assertFalse(future.isCancelled()); + Assert.assertTrue(future.isDone()); + Assert.assertEquals(1, future.jobIds().size()); + } + @Test public void foreachAsync() throws Exception { List data = Arrays.asList(1, 2, 3, 4, 5); diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 50f347f1954de..b8e3e83b5a47b 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -79,26 +79,49 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { val byteArray2 = converter.convert(bytesWritable) assert(byteArray2.length === 0) } - + test("addFile works") { - val file = File.createTempFile("someprefix", "somesuffix") - val absolutePath = file.getAbsolutePath + val file1 = File.createTempFile("someprefix1", "somesuffix1") + val absolutePath1 = file1.getAbsolutePath + + val pluto = Utils.createTempDir() + val file2 = File.createTempFile("someprefix2", "somesuffix2", pluto) + val relativePath = file2.getParent + "/../" + file2.getParentFile.getName + "/" + file2.getName + val absolutePath2 = file2.getAbsolutePath + try { - Files.write("somewords", file, UTF_8) - val length = file.length() + Files.write("somewords1", file1, UTF_8) + Files.write("somewords2", file2, UTF_8) + val length1 = file1.length() + val length2 = file2.length() + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) - sc.addFile(file.getAbsolutePath) + sc.addFile(file1.getAbsolutePath) + sc.addFile(relativePath) sc.parallelize(Array(1), 1).map(x => { - val gotten = new File(SparkFiles.get(file.getName)) - if (!gotten.exists()) { - throw new SparkException("file doesn't exist") + val gotten1 = new File(SparkFiles.get(file1.getName)) + val gotten2 = new File(SparkFiles.get(file2.getName)) + if (!gotten1.exists()) { + throw new SparkException("file doesn't exist : " + absolutePath1) + } + if (!gotten2.exists()) { + throw new SparkException("file doesn't exist : " + absolutePath2) } - if (length != gotten.length()) { + + if (length1 != gotten1.length()) { + throw new SparkException( + s"file has different length $length1 than added file ${gotten1.length()} : " + absolutePath1) + } + if (length2 != gotten2.length()) { throw new SparkException( - s"file has different length $length than added file ${gotten.length()}") + s"file has different length $length2 than added file ${gotten2.length()} : " + absolutePath2) } - if (absolutePath == gotten.getAbsolutePath) { - throw new SparkException("file should have been copied") + + if (absolutePath1 == gotten1.getAbsolutePath) { + throw new SparkException("file should have been copied :" + absolutePath1) + } + if (absolutePath2 == gotten2.getAbsolutePath) { + throw new SparkException("file should have been copied : " + absolutePath2) } x }).count() @@ -106,7 +129,7 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { sc.stop() } } - + test("addFile recursive works") { val pluto = Utils.createTempDir() val neptune = Utils.createTempDir(pluto.getAbsolutePath) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 30119ce5d4eec..63360a0f189a3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -322,6 +322,18 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assertDataStructuresEmpty } + test("regression test for getCacheLocs") { + val rdd = new MyRDD(sc, 3, Nil) + cacheLocations(rdd.id -> 0) = + Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")) + cacheLocations(rdd.id -> 1) = + Seq(makeBlockManagerId("hostB"), makeBlockManagerId("hostC")) + cacheLocations(rdd.id -> 2) = + Seq(makeBlockManagerId("hostC"), makeBlockManagerId("hostD")) + val locs = scheduler.getCacheLocs(rdd).map(_.map(_.host)) + assert(locs === Seq(Seq("hostA", "hostB"), Seq("hostB", "hostC"), Seq("hostC", "hostD"))) + } + test("avoid exponential blowup when getting preferred locs list") { // Build up a complex dependency graph with repeated zip operations, without preferred locations. var rdd: RDD[_] = new MyRDD(sc, 1, Nil) diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 992dde66f982f..448258a754153 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -25,9 +25,9 @@ import scala.io.Source import org.apache.hadoop.fs.Path import org.json4s.jackson.JsonMethods._ -import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.{FunSuiteLike, BeforeAndAfter, FunSuite} -import org.apache.spark.{Logging, SparkConf, SparkContext, SPARK_VERSION} +import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io._ import org.apache.spark.util.{JsonProtocol, Utils} @@ -39,7 +39,8 @@ import org.apache.spark.util.{JsonProtocol, Utils} * logging events, whether the parsing of the file names is correct, and whether the logged events * can be read and deserialized into actual SparkListenerEvents. */ -class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Logging { +class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with BeforeAndAfter + with Logging { import EventLoggingListenerSuite._ private val fileSystem = Utils.getHadoopFileSystem("/", @@ -144,7 +145,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin // A comprehensive test on JSON de/serialization of all events is in JsonProtocolSuite eventLogger.start() - listenerBus.start() + listenerBus.start(sc) listenerBus.addListener(eventLogger) listenerBus.postToAll(applicationStart) listenerBus.postToAll(applicationEnd) diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index 3cc860caa1d9b..c8c957856247a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -153,7 +153,7 @@ class OutputCommitCoordinatorSuite extends FunSuite with BeforeAndAfter { def resultHandler(x: Int, y: Unit): Unit = {} val futureAction: SimpleFutureAction[Unit] = sc.submitJob[Int, Unit, Unit](rdd, OutputCommitFunctions(tempDir.getAbsolutePath).commitSuccessfully, - 0 until rdd.partitions.size, resultHandler, 0) + 0 until rdd.partitions.size, resultHandler, () => Unit) // It's an error if the job completes successfully even though no committer was authorized, // so throw an exception if the job was allowed to complete. intercept[TimeoutException] { diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 3a41ee8d4ae0c..627c9a4ddfffc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -46,7 +46,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers assert(counter.count === 0) // Starting listener bus should flush all buffered events - bus.start() + bus.start(sc) assert(bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(counter.count === 5) @@ -58,8 +58,8 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers // Listener bus must not be started twice intercept[IllegalStateException] { val bus = new LiveListenerBus - bus.start() - bus.start() + bus.start(sc) + bus.start(sc) } // ... or stopped before starting @@ -96,7 +96,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers val blockingListener = new BlockingListener bus.addListener(blockingListener) - bus.start() + bus.start(sc) bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) listenerStarted.acquire() @@ -347,7 +347,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers bus.addListener(badListener) bus.addListener(jobCounter1) bus.addListener(jobCounter2) - bus.start() + bus.start(sc) // Post events to all listeners, and wait until the queue is drained (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 6a972381faf14..0d155982a8c54 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -17,20 +17,24 @@ package org.apache.spark.ui +import javax.servlet.http.HttpServletRequest + import scala.collection.JavaConversions._ +import scala.xml.Node -import org.openqa.selenium.{By, WebDriver} import org.openqa.selenium.htmlunit.HtmlUnitDriver +import org.openqa.selenium.{By, WebDriver} import org.scalatest._ import org.scalatest.concurrent.Eventually._ import org.scalatest.selenium.WebBrowser import org.scalatest.time.SpanSugar._ -import org.apache.spark._ import org.apache.spark.LocalSparkContext._ +import org.apache.spark._ import org.apache.spark.api.java.StorageLevels import org.apache.spark.shuffle.FetchFailedException + /** * Selenium tests for the Spark Web UI. */ @@ -310,4 +314,46 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before } } } + + test("attaching and detaching a new tab") { + withSpark(newSparkContext()) { sc => + val sparkUI = sc.ui.get + + val newTab = new WebUITab(sparkUI, "foo") { + attachPage(new WebUIPage("") { + def render(request: HttpServletRequest): Seq[Node] = { + "html magic" + } + }) + } + sparkUI.attachTab(newTab) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/")) + find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None) + find(cssSelector("""ul li a[href*="stages"]""")) should not be(None) + find(cssSelector("""ul li a[href*="storage"]""")) should not be(None) + find(cssSelector("""ul li a[href*="environment"]""")) should not be(None) + find(cssSelector("""ul li a[href*="foo"]""")) should not be(None) + } + eventually(timeout(10 seconds), interval(50 milliseconds)) { + // check whether new page exists + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo") + find(cssSelector("b")).get.text should include ("html magic") + } + sparkUI.detachTab(newTab) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/")) + find(cssSelector("""ul li a[href*="jobs"]""")) should not be(None) + find(cssSelector("""ul li a[href*="stages"]""")) should not be(None) + find(cssSelector("""ul li a[href*="storage"]""")) should not be(None) + find(cssSelector("""ul li a[href*="environment"]""")) should not be(None) + find(cssSelector("""ul li a[href*="foo"]""")) should be(None) + } + eventually(timeout(10 seconds), interval(50 milliseconds)) { + // check new page not exist + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/foo") + find(cssSelector("b")) should be(None) + } + } + } } diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 92a21f82f3c21..77a038dc1720d 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.ui import java.net.ServerSocket -import javax.servlet.http.HttpServletRequest import scala.io.Source import scala.util.{Failure, Success, Try} @@ -28,9 +27,8 @@ import org.scalatest.FunSuite import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.{SparkContext, SparkConf} import org.apache.spark.LocalSparkContext._ -import scala.xml.Node +import org.apache.spark.{SparkConf, SparkContext} class UISuite extends FunSuite { @@ -72,40 +70,6 @@ class UISuite extends FunSuite { } } - ignore("attaching a new tab") { - withSpark(newSparkContext()) { sc => - val sparkUI = sc.ui.get - - val newTab = new WebUITab(sparkUI, "foo") { - attachPage(new WebUIPage("") { - def render(request: HttpServletRequest): Seq[Node] = { - "html magic" - } - }) - } - sparkUI.attachTab(newTab) - eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(sparkUI.appUIAddress).mkString - assert(!html.contains("random data that should not be present")) - - // check whether new page exists - assert(html.toLowerCase.contains("foo")) - - // check whether other pages still exist - assert(html.toLowerCase.contains("stages")) - assert(html.toLowerCase.contains("storage")) - assert(html.toLowerCase.contains("environment")) - assert(html.toLowerCase.contains("executors")) - } - - eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(sparkUI.appUIAddress.stripSuffix("/") + "/foo").mkString - // check whether new page exists - assert(html.contains("magic")) - } - } - } - test("jetty selects different port under contention") { val server = new ServerSocket(0) val startPort = server.getLocalPort diff --git a/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala index 31e3b7e7bb71b..87de90bb0dfb0 100644 --- a/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala @@ -21,8 +21,7 @@ import java.net.URLClassLoader import org.scalatest.FunSuite -import org.apache.spark.{LocalSparkContext, SparkContext, SparkException, TestUtils} -import org.apache.spark.util.Utils +import org.apache.spark.{SparkContext, SparkException, TestUtils} class MutableURLClassLoaderSuite extends FunSuite { diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 6f87fcd6d4eb4..b5a67dd783b93 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -116,6 +116,8 @@ if [[ ! "$@" =~ --skip-publish ]]; then staged_repo_id=$(echo $out | sed -e "s/.*\(orgapachespark-[0-9]\{4\}\).*/\1/") echo "Created Nexus staging repository: $staged_repo_id" + rm -rf $SPARK_REPO + build/mvn -DskipTests -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ clean install @@ -128,7 +130,6 @@ if [[ ! "$@" =~ --skip-publish ]]; then ./dev/change-version-to-2.10.sh - rm -rf $SPARK_REPO pushd $SPARK_REPO # Remove any extra files generated during install @@ -160,7 +161,7 @@ if [[ ! "$@" =~ --skip-publish ]]; then done echo "Closing nexus staging repository" - repo_request="$staged_repo_idApache Spark $GIT_TAG" + repo_request="$staged_repo_idApache Spark $GIT_TAG (published as $PUBLISH_VERSION)" out=$(curl -X POST -d "$repo_request" -u $ASF_USERNAME:$ASF_PASSWORD \ -H "Content-Type:application/xml" -v \ $NEXUS_ROOT/profiles/$NEXUS_PROFILE/finish) @@ -192,10 +193,12 @@ if [[ ! "$@" =~ --skip-package ]]; then echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md SHA512 spark-$RELEASE_VERSION.tgz > \ spark-$RELEASE_VERSION.tgz.sha rm -rf spark-$RELEASE_VERSION - + + # Updated for each binary build make_binary_release() { NAME=$1 FLAGS=$2 + ZINC_PORT=$3 cp -r spark spark-$RELEASE_VERSION-bin-$NAME cd spark-$RELEASE_VERSION-bin-$NAME @@ -205,16 +208,12 @@ if [[ ! "$@" =~ --skip-package ]]; then ./dev/change-version-to-2.11.sh fi - # Create new Zinc instances for each binary release to avoid interference - # that causes OOM's and random compiler crashes. - zinc_port=${zinc_port:-3030} - zinc_port=$[$zinc_port + 1] - export ZINC_PORT=$zinc_port - - ./make-distribution.sh --name $NAME --tgz $FLAGS 2>&1 | tee ../binary-release-$NAME.log + export ZINC_PORT=$ZINC_PORT + echo "Creating distribution: $NAME ($FLAGS)" + ./make-distribution.sh --name $NAME --tgz $FLAGS -DzincPort=$ZINC_PORT 2>&1 > \ + ../binary-release-$NAME.log cd .. cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . - rm -rf spark-$RELEASE_VERSION-bin-$NAME echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour \ --output spark-$RELEASE_VERSION-bin-$NAME.tgz.asc \ @@ -227,16 +226,18 @@ if [[ ! "$@" =~ --skip-package ]]; then spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } - - make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" & - make_binary_release "hadoop1-scala2.11" "-Phive -Dscala-2.11" & - make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & - make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" & - make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" & - make_binary_release "mapr3" "-Pmapr3 -Phive -Phive-thriftserver" & - make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive -Phive-thriftserver" & - make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" & + # We increment the Zinc port each time to avoid OOM's and other craziness if multiple builds + # share the same Zinc server. + make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" "3030" & + make_binary_release "hadoop1-scala2.11" "-Phive -Dscala-2.11" "3031" & + make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" "3032" & + make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" "3033" & + make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" "3034" & + make_binary_release "mapr3" "-Pmapr3 -Phive -Phive-thriftserver" "3035" & + make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive -Phive-thriftserver" "3036" & + make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" "3037" & wait + rm -rf spark-$RELEASE_VERSION-bin-*/ # Copy data echo "Copying release tarballs" diff --git a/dev/create-release/known_translations b/dev/create-release/known_translations index b74e4ee8a330b..0a599b5a65549 100644 --- a/dev/create-release/known_translations +++ b/dev/create-release/known_translations @@ -57,3 +57,37 @@ watermen - Yadong Qi witgo - Guoqiang Li xinyunh - Xinyun Huang zsxwing - Shixiong Zhu +Bilna - Bilna P +DoingDone9 - Doing Done +Earne - Ernest +FlytxtRnD - Meethu Mathew +GenTang - Gen TANG +JoshRosen - Josh Rosen +MechCoder - Manoj Kumar +OopsOutOfMemory - Sheng Li +Peishen-Jia - Peishen Jia +SaintBacchus - Huang Zhaowei +azagrebin - Andrey Zagrebin +bzz - Alexander Bezzubov +fjiang6 - Fan Jiang +gasparms - Gaspar Munoz +guowei2 - Guo Wei +hhbyyh - Yuhao Yang +hseagle - Peng Xu +javadba - Stephen Boesch +jbencook - Ben Cook +kul - Kuldeep +ligangty - Gang Li +marsishandsome - Liangliang Gu +medale - Markus Dale +nemccarthy - Nathan McCarthy +nxwhite-str - Nate Crosswhite +seayi - Xiaohua Yi +tianyi - Yi Tian +uncleGen - Uncle Gen +viper-kun - Xu Kun +x1- - Yuri Saito +zapletal-martin - Martin Zapletal +zuxqoj - Shekhar Bansal +mingyukim - Mingyu Kim +sigmoidanalytics - Mayur Rustagi diff --git a/dev/lint-python b/dev/lint-python index 772f856154ae0..fded654893a7c 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -19,43 +19,53 @@ SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" SPARK_ROOT_DIR="$(dirname "$SCRIPT_DIR")" -PEP8_REPORT_PATH="$SPARK_ROOT_DIR/dev/pep8-report.txt" +PATHS_TO_CHECK="./python/pyspark/ ./ec2/spark_ec2.py ./examples/src/main/python/" +PYTHON_LINT_REPORT_PATH="$SPARK_ROOT_DIR/dev/python-lint-report.txt" cd "$SPARK_ROOT_DIR" +# compileall: https://docs.python.org/2/library/compileall.html +python -B -m compileall -q -l $PATHS_TO_CHECK > "$PYTHON_LINT_REPORT_PATH" +compile_status="${PIPESTATUS[0]}" + # Get pep8 at runtime so that we don't rely on it being installed on the build server. #+ See: https://github.com/apache/spark/pull/1744#issuecomment-50982162 #+ TODOs: -#+ - Dynamically determine latest release version of pep8 and use that. -#+ - Download this from a more reliable source. (GitHub raw can be flaky, apparently. (?)) +#+ - Download pep8 from PyPI. It's more "official". PEP8_SCRIPT_PATH="$SPARK_ROOT_DIR/dev/pep8.py" -PEP8_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/jcrocholl/pep8/1.5.7/pep8.py" -PEP8_PATHS_TO_CHECK="./python/pyspark/ ./ec2/spark_ec2.py ./examples/src/main/python/" +PEP8_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/jcrocholl/pep8/1.6.2/pep8.py" +# if [ ! -e "$PEP8_SCRIPT_PATH" ]; then curl --silent -o "$PEP8_SCRIPT_PATH" "$PEP8_SCRIPT_REMOTE_PATH" -curl_status=$? +curl_status="$?" -if [ $curl_status -ne 0 ]; then +if [ "$curl_status" -ne 0 ]; then echo "Failed to download pep8.py from \"$PEP8_SCRIPT_REMOTE_PATH\"." - exit $curl_status + exit "$curl_status" fi - +# fi # There is no need to write this output to a file #+ first, but we do so so that the check status can #+ be output before the report, like with the #+ scalastyle and RAT checks. -python "$PEP8_SCRIPT_PATH" $PEP8_PATHS_TO_CHECK > "$PEP8_REPORT_PATH" -pep8_status=${PIPESTATUS[0]} #$? +python "$PEP8_SCRIPT_PATH" --ignore=E402,E731,E241,W503,E226 $PATHS_TO_CHECK >> "$PYTHON_LINT_REPORT_PATH" +pep8_status="${PIPESTATUS[0]}" + +if [ "$compile_status" -eq 0 -a "$pep8_status" -eq 0 ]; then + lint_status=0 +else + lint_status=1 +fi -if [ $pep8_status -ne 0 ]; then - echo "PEP 8 checks failed." - cat "$PEP8_REPORT_PATH" +if [ "$lint_status" -ne 0 ]; then + echo "Python lint checks failed." + cat "$PYTHON_LINT_REPORT_PATH" else - echo "PEP 8 checks passed." + echo "Python lint checks passed." fi -rm "$PEP8_REPORT_PATH" rm "$PEP8_SCRIPT_PATH" +rm "$PYTHON_LINT_REPORT_PATH" -exit $pep8_status +exit "$lint_status" diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 6a849e4f77207..5f4000e83925c 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -49,6 +49,21 @@ SHORT_COMMIT_HASH="${ghprbActualCommit:0:7}" TESTS_TIMEOUT="120m" # format: http://linux.die.net/man/1/timeout +# Array to capture all tests to run on the pull request. These tests are held under the +#+ dev/tests/ directory. +# +# To write a PR test: +#+ * the file must reside within the dev/tests directory +#+ * be an executable bash script +#+ * accept two arguments on the command line, the first being the Github PR long commit +#+ hash and the second the Github SHA1 hash +#+ * and, lastly, return string output to be included in the pr message output that will +#+ be posted to Github +PR_TESTS=( + "pr_merge_ability" + "pr_public_classes" +) + function post_message () { local message=$1 local data="{\"body\": \"$message\"}" @@ -131,48 +146,22 @@ function send_archived_logs () { fi } - -# We diff master...$ghprbActualCommit because that gets us changes introduced in the PR -#+ and not anything else added to master since the PR was branched. - -# check PR merge-ability and check for new public classes -{ - if [ "$sha1" == "$ghprbActualCommit" ]; then - merge_note=" * This patch **does not merge cleanly**." - else - merge_note=" * This patch merges cleanly." +# Environment variable to capture PR test output +pr_message="" + +# Run pull request tests +for t in "${PR_TESTS[@]}"; do + this_test="${FWDIR}/dev/tests/${t}.sh" + # Ensure the test is a file and is executable + if [ -x "$this_test" ]; then + echo "ghprb: $ghprbActualCommit sha1: $sha1" + this_mssg="`bash \"${this_test}\" \"${ghprbActualCommit}\" \"${sha1}\" 2>/dev/null`" + # Check if this is the merge test as we submit that note *before* and *after* + # the tests run + [ "$t" == "pr_merge_ability" ] && merge_note="${this_mssg}" + pr_message="${pr_message}\n${this_mssg}" fi - - source_files=$( - git diff master...$ghprbActualCommit --name-only `# diff patch against master from branch point` \ - | grep -v -e "\/test" `# ignore files in test directories` \ - | grep -e "\.py$" -e "\.java$" -e "\.scala$" `# include only code files` \ - | tr "\n" " " - ) - new_public_classes=$( - git diff master...$ghprbActualCommit ${source_files} `# diff patch against master from branch point` \ - | grep "^\+" `# filter in only added lines` \ - | sed -r -e "s/^\+//g" `# remove the leading +` \ - | grep -e "trait " -e "class " `# filter in lines with these key words` \ - | grep -e "{" -e "(" `# filter in lines with these key words, too` \ - | grep -v -e "\@\@" -e "private" `# exclude lines with these words` \ - | grep -v -e "^// " -e "^/\*" -e "^ \* " `# exclude comment lines` \ - | sed -r -e "s/\{.*//g" `# remove from the { onwards` \ - | sed -r -e "s/\}//g" `# just in case, remove }; they mess the JSON` \ - | sed -r -e "s/\"/\\\\\"/g" `# escape double quotes; they mess the JSON` \ - | sed -r -e "s/^(.*)$/\`\1\`/g" `# surround with backticks for style` \ - | sed -r -e "s/^/ \* /g" `# prepend ' *' to start of line` \ - | sed -r -e "s/$/\\\n/g" `# append newline to end of line` \ - | tr -d "\n" `# remove actual LF characters` - ) - - if [ -z "$new_public_classes" ]; then - public_classes_note=" * This patch adds no public classes." - else - public_classes_note=" * This patch adds the following public classes _(experimental)_:" - public_classes_note="${public_classes_note}\n${new_public_classes}" - fi -} +done # post start message { @@ -181,7 +170,6 @@ function send_archived_logs () { PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." start_message="${start_message}\n${merge_note}" - # start_message="${start_message}\n${public_classes_note}" post_message "$start_message" } @@ -234,8 +222,7 @@ function send_archived_logs () { PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." result_message="${result_message}\n${test_result_note}" - result_message="${result_message}\n${merge_note}" - result_message="${result_message}\n${public_classes_note}" + result_message="${result_message}\n${pr_message}" post_message "$result_message" } diff --git a/dev/tests/pr_merge_ability.sh b/dev/tests/pr_merge_ability.sh new file mode 100755 index 0000000000000..d9a347fe24a8c --- /dev/null +++ b/dev/tests/pr_merge_ability.sh @@ -0,0 +1,39 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# +# This script follows the base format for testing pull requests against +# another branch and returning results to be published. More details can be +# found at dev/run-tests-jenkins. +# +# Arg1: The Github Pull Request Actual Commit +#+ known as `ghprbActualCommit` in `run-tests-jenkins` +# Arg2: The SHA1 hash +#+ known as `sha1` in `run-tests-jenkins` +# + +ghprbActualCommit="$1" +sha1="$2" + +# check PR merge-ability +if [ "${sha1}" == "${ghprbActualCommit}" ]; then + echo " * This patch **does not merge cleanly**." +else + echo " * This patch merges cleanly." +fi diff --git a/dev/tests/pr_public_classes.sh b/dev/tests/pr_public_classes.sh new file mode 100755 index 0000000000000..927295b88c963 --- /dev/null +++ b/dev/tests/pr_public_classes.sh @@ -0,0 +1,65 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# +# This script follows the base format for testing pull requests against +# another branch and returning results to be published. More details can be +# found at dev/run-tests-jenkins. +# +# Arg1: The Github Pull Request Actual Commit +#+ known as `ghprbActualCommit` in `run-tests-jenkins` +# Arg2: The SHA1 hash +#+ known as `sha1` in `run-tests-jenkins` +# + +# We diff master...$ghprbActualCommit because that gets us changes introduced in the PR +#+ and not anything else added to master since the PR was branched. + +ghprbActualCommit="$1" +sha1="$2" + +source_files=$( + git diff master...$ghprbActualCommit --name-only `# diff patch against master from branch point` \ + | grep -v -e "\/test" `# ignore files in test directories` \ + | grep -e "\.py$" -e "\.java$" -e "\.scala$" `# include only code files` \ + | tr "\n" " " +) +new_public_classes=$( + git diff master...$ghprbActualCommit ${source_files} `# diff patch against master from branch point` \ + | grep "^\+" `# filter in only added lines` \ + | sed -r -e "s/^\+//g" `# remove the leading +` \ + | grep -e "trait " -e "class " `# filter in lines with these key words` \ + | grep -e "{" -e "(" `# filter in lines with these key words, too` \ + | grep -v -e "\@\@" -e "private" `# exclude lines with these words` \ + | grep -v -e "^// " -e "^/\*" -e "^ \* " `# exclude comment lines` \ + | sed -r -e "s/\{.*//g" `# remove from the { onwards` \ + | sed -r -e "s/\}//g" `# just in case, remove }; they mess the JSON` \ + | sed -r -e "s/\"/\\\\\"/g" `# escape double quotes; they mess the JSON` \ + | sed -r -e "s/^(.*)$/\`\1\`/g" `# surround with backticks for style` \ + | sed -r -e "s/^/ \* /g" `# prepend ' *' to start of line` \ + | sed -r -e "s/$/\\\n/g" `# append newline to end of line` \ + | tr -d "\n" `# remove actual LF characters` +) + +if [ -z "$new_public_classes" ]; then + echo " * This patch adds no public classes." +else + public_classes_note=" * This patch adds the following public classes _(experimental)_:" + echo "${public_classes_note}\n${new_public_classes}" +fi diff --git a/docs/building-spark.md b/docs/building-spark.md index 57d0ca834f460..ea79c5bc276d3 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -23,6 +23,18 @@ build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package Other build examples can be found below. +**Note:** When building on an encrypted filesystem (if your home directory is encrypted, for example), then the Spark build might fail with a "Filename too long" error. As a workaround, add the following in the configuration args of the `scala-maven-plugin` in the project `pom.xml`: + + -Xmax-classfile-name + 128 + +and in `project/SparkBuild.scala` add: + + scalacOptions in Compile ++= Seq("-Xmax-classfile-name", "128"), + +to the `sharedSettings` val. See also [this PR](https://github.com/apache/spark/pull/2883/files) if you are unsure of where to add these lines. + + # Setting up Maven's Memory Usage You'll need to configure Maven to use more memory than usual by setting `MAVEN_OPTS`. We recommend the following settings: diff --git a/docs/configuration.md b/docs/configuration.md index ae90fe1f8f6b9..7fe11475212b3 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -745,6 +745,18 @@ Apart from these, the following properties are also available, and may be useful the driver, in seconds. + + spark.files.useFetchCache + true + + If set to true (default), file fetching will use a local cache that is shared by executors + that belong to the same application, which can improve task launching performance when + running many executors on the same host. If set to false, these caching optimizations will + be disabled and all executors will fetch their own copies of files. This optimization may be + disabled in order to use Spark local directories that reside on NFS filesystems (see + SPARK-6313 for more details). + + spark.files.overwrite false @@ -1345,9 +1357,9 @@ Apart from these, the following properties are also available, and may be useful spark.streaming.receiver.maxRate - infinite + not set - Maximum number records per second at which each receiver will receive data. + Maximum rate (number of records per second) at which each receiver will receive data. Effectively, each stream will consume at most this number of records per second. Setting this configuration to 0 or a negative number will put no limit on the rate. See the deployment guide @@ -1375,15 +1387,27 @@ Apart from these, the following properties are also available, and may be useful higher memory usage in Spark. + + spark.streaming.kafka.maxRatePerPartition + not set + + Maximum rate (number of records per second) at which data will be read from each Kafka + partition when using the new Kafka direct stream API. See the + Kafka Integration guide + for more details. + + #### Cluster Managers Each cluster manager in Spark has additional configuration options. Configurations can be found on the pages for each mode: - * [YARN](running-on-yarn.html#configuration) - * [Mesos](running-on-mesos.html) - * [Standalone Mode](spark-standalone.html#cluster-launch-scripts) +##### [YARN](running-on-yarn.html#configuration) + +##### [Mesos](running-on-mesos.html#configuration) + +##### [Standalone Mode](spark-standalone.html#cluster-launch-scripts) # Environment Variables diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index 8c9a1e1262d8f..7f60f82b966fe 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -5,7 +5,7 @@ title: Running Spark on EC2 The `spark-ec2` script, located in Spark's `ec2` directory, allows you to launch, manage and shut down Spark clusters on Amazon EC2. It automatically -sets up Spark, Shark and HDFS on the cluster for you. This guide describes +sets up Spark and HDFS on the cluster for you. This guide describes how to use `spark-ec2` to launch clusters, how to run jobs on them, and how to shut them down. It assumes you've already signed up for an EC2 account on the [Amazon Web Services site](http://aws.amazon.com/). diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index 5295e351dd711..963e88a3e1d8f 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -14,8 +14,7 @@ runs an independent set of executor processes. The cluster managers that Spark r facilities for [scheduling across applications](#scheduling-across-applications). Second, _within_ each Spark application, multiple "jobs" (Spark actions) may be running concurrently if they were submitted by different threads. This is common if your application is serving requests -over the network; for example, the [Shark](http://shark.cs.berkeley.edu) server works this way. Spark -includes a [fair scheduler](#scheduling-within-an-application) to schedule resources within each SparkContext. +over the network. Spark includes a [fair scheduler](#scheduling-within-an-application) to schedule resources within each SparkContext. # Scheduling Across Applications @@ -52,8 +51,7 @@ an application to gain back cores on one node when it has work to do. To use thi Note that none of the modes currently provide memory sharing across applications. If you would like to share data this way, we recommend running a single server application that can serve multiple requests by querying -the same RDDs. For example, the [Shark](http://shark.cs.berkeley.edu) JDBC server works this way for SQL -queries. In future releases, in-memory storage systems such as [Tachyon](http://tachyon-project.org) will +the same RDDs. In future releases, in-memory storage systems such as [Tachyon](http://tachyon-project.org) will provide another approach to share RDDs. ## Dynamic Resource Allocation diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 4c7a7d9115ca1..f8e879496c135 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -80,11 +80,6 @@ include `netlib-java`'s native proxies by default. To configure [netlib-java](https://github.com/fommil/netlib-java) documentation for your platform's additional installation instructions. -MLlib also uses [jblas](https://github.com/mikiobraun/jblas) which -will require you to install the -[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) -if it is not already present on your nodes. - To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. @@ -107,6 +102,8 @@ In the `spark.mllib` package, there were several breaking changes. The first ch * In `DecisionTree`, the deprecated class method `train` has been removed. (The object/static `train` methods remain.) * In `Strategy`, the `checkpointDir` parameter has been removed. Checkpointing is still supported, but the checkpoint directory must be set before calling tree and tree ensemble training. * `PythonMLlibAPI` (the interface between Scala/Java and Python for MLlib) was a public API but is now private, declared `private[python]`. This was never meant for external use. +* In linear regression (including Lasso and ridge regression), the squared loss is now divided by 2. + So in order to produce the same result as in 1.2, the regularization parameter needs to be divided by 2 and the step size needs to be multiplied by 2. ## Previous Spark Versions diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index 4d101afca2c97..6cabc1610a151 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -203,6 +203,10 @@ regularization, as well as L2 regularizer. recommended. * `maxNumIterations` is the maximal number of iterations that L-BFGS can be run. * `regParam` is the regularization parameter when using regularization. +* `convergenceTol` controls how much relative change is still allowed when L-BFGS +is considered to converge. This must be nonnegative. Lower values are less tolerant and +therefore generally cause more iterations to be run. This value looks at both average +improvement and the norm of gradient inside [Breeze LBFGS](https://github.com/scalanlp/breeze/blob/master/math/src/main/scala/breeze/optimize/LBFGS.scala). The `return` is a tuple containing two elements. The first element is a column matrix containing weights for every feature, and the second element is an array containing diff --git a/docs/programming-guide.md b/docs/programming-guide.md index fa0b4e3705d6e..eda3a95426182 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -725,7 +725,7 @@ class MyClass(object): def __init__(self): self.field = "Hello" def doStuff(self, rdd): - return rdd.map(lambda s: self.field + x) + return rdd.map(lambda s: self.field + s) {% endhighlight %} To avoid this issue, the simplest way is to copy `field` into a local variable instead @@ -734,13 +734,76 @@ of accessing it externally: {% highlight python %} def doStuff(self, rdd): field = self.field - return rdd.map(lambda s: field + x) + return rdd.map(lambda s: field + s) {% endhighlight %} +### Understanding closures +One of the harder things about Spark is understanding the scope and life cycle of variables and methods when executing code across a cluster. RDD operations that modify variables outside of their scope can be a frequent source of confusion. In the example below we'll look at code that uses `foreach()` to increment a counter, but similar issues can occur for other operations as well. + +#### Example + +Consider the naive RDD element sum below, which behaves completely differently depending on whether execution is happening within the same JVM. A common example of this is when running Spark in `local` mode (`--master = local[n]`) versus deploying a Spark application to a cluster (e.g. via spark-submit to YARN): + +
+ +
+{% highlight scala %} +var counter = 0 +var rdd = sc.parallelize(data) + +// Wrong: Don't do this!! +rdd.foreach(x => counter += x) + +println("Counter value: " + counter) +{% endhighlight %} +
+ +
+{% highlight java %} +int counter = 0; +JavaRDD rdd = sc.parallelize(data); + +// Wrong: Don't do this!! +rdd.foreach(x -> counter += x); + +println("Counter value: " + counter); +{% endhighlight %} +
+ +
+{% highlight python %} +counter = 0 +rdd = sc.parallelize(data) + +# Wrong: Don't do this!! +rdd.foreach(lambda x: counter += x) + +print("Counter value: " + counter) + +{% endhighlight %} +
+ +
+ +#### Local vs. cluster modes + +The primary challenge is that the behavior of the above code is undefined. In local mode with a single JVM, the above code will sum the values within the RDD and store it in **counter**. This is because both the RDD and the variable **counter** are in the same memory space on the driver node. + +However, in `cluster` mode, what happens is more complicated, and the above may not work as intended. To execute jobs, Spark breaks up the processing of RDD operations into tasks - each of which is operated on by an executor. Prior to execution, Spark computes the **closure**. The closure is those variables and methods which must be visible for the executor to perform its computations on the RDD (in this case `foreach()`). This closure is serialized and sent to each executor. In `local` mode, there is only the one executors so everything shares the same closure. In other modes however, this is not the case and the executors running on seperate worker nodes each have their own copy of the closure. + +What is happening here is that the variables within the closure sent to each executor are now copies and thus, when **counter** is referenced within the `foreach` function, it's no longer the **counter** on the driver node. There is still a **counter** in the memory of the driver node but this is no longer visible to the executors! The executors only sees the copy from the serialized closure. Thus, the final value of **counter** will still be zero since all operations on **counter** were referencing the value within the serialized closure. + +To ensure well-defined behavior in these sorts of scenarios one should use an [`Accumulator`](#AccumLink). Accumulators in Spark are used specifically to provide a mechanism for safely updating a variable when execution is split up across worker nodes in a cluster. The Accumulators section of this guide discusses these in more detail. + +In general, closures - constructs like loops or locally defined methods, should not be used to mutate some global state. Spark does not define or guarantee the behavior of mutations to objects referenced from outside of closures. Some code that does this may work in local mode, but that's just by accident and such code will not behave as expected in distributed mode. Use an Accumulator instead if some global aggregation is needed. + +#### Printing elements of an RDD +Another common idiom is attempting to print out the elements of an RDD using `rdd.foreach(println)` or `rdd.map(println)`. On a single machine, this will generate the expected output and print all the RDD's elements. However, in `cluster` mode, the output to `stdout` being called by the executors is now writing to the executor's `stdout` instead, not the one on the driver, so `stdout` on the driver won't show these! To print all elements on the driver, one can use the `collect()` method to first bring the RDD to the driver node thus: `rdd.collect().foreach(println)`. This can cause the driver to run out of memory, though, because `collect()` fetches the entire RDD to a single machine; if you only need to print a few elements of the RDD, a safer approach is to use the `take()`: `rdd.take(100).foreach(println)`. + ### Working with Key-Value Pairs
@@ -1018,7 +1081,8 @@ for details. foreach(func) - Run a function func on each element of the dataset. This is usually done for side effects such as updating an accumulator variable (see below) or interacting with external storage systems. + Run a function func on each element of the dataset. This is usually done for side effects such as updating an Accumulator or interacting with external storage systems. +
Note: modifying variables other than Accumulators outside of the foreach() may result in undefined behavior. See Understanding closures for more details. @@ -1143,6 +1207,12 @@ than shipping a copy of it with tasks. They can be used, for example, to give ev large input dataset in an efficient manner. Spark also attempts to distribute broadcast variables using efficient broadcast algorithms to reduce communication cost. +Spark actions are executed through a set of stages, separated by distributed "shuffle" operations. +Spark automatically broadcasts the common data needed by tasks within each stage. The data +broadcasted this way is cached in serialized form and deserialized before running each task. This +means that explicitly creating broadcast variables is only useful when tasks across multiple stages +need the same data or when caching the data in deserialized form is important. + Broadcast variables are created from a variable `v` by calling `SparkContext.broadcast(v)`. The broadcast variable is a wrapper around `v`, and its value can be accessed by calling the `value` method. The code below shows this: @@ -1191,7 +1261,7 @@ run on the cluster so that `v` is not shipped to the nodes more than once. In ad `v` should not be modified after it is broadcast in order to ensure that all nodes get the same value of the broadcast variable (e.g. if the variable is shipped to a new node later). -## Accumulators +## Accumulators Accumulators are variables that are only "added" to through an associative operation and can therefore be efficiently supported in parallel. They can be used to implement counters (as in diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index e509e4bf37396..6a9d304501dc0 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -110,7 +110,7 @@ cluster, or `mesos://zk://host:2181` for a multi-master Mesos cluster using ZooK The driver also needs some configuration in `spark-env.sh` to interact properly with Mesos: 1. In `spark-env.sh` set some environment variables: - * `export MESOS_NATIVE_LIBRARY=`. This path is typically + * `export MESOS_NATIVE_JAVA_LIBRARY=`. This path is typically `/lib/libmesos.so` where the prefix is `/usr/local` by default. See Mesos installation instructions above. On Mac OS X, the library is called `libmesos.dylib` instead of `libmesos.so`. @@ -167,9 +167,6 @@ acquire. By default, it will acquire *all* cores in the cluster (that get offere only makes sense if you run just one application at a time. You can cap the maximum number of cores using `conf.set("spark.cores.max", "10")` (for example). -# Known issues -- When using the "fine-grained" mode, make sure that your executors always leave 32 MB free on the slaves. Otherwise it can happen that your Spark job does not proceed anymore. Currently, Apache Mesos only offers resources if there are at least 32 MB memory allocatable. But as Spark allocates memory only for the executor and cpu only for tasks, it can happen on high slave memory usage that no new tasks will be started anymore. More details can be found in [MESOS-1688](https://issues.apache.org/jira/browse/MESOS-1688). Alternatively use the "coarse-gained" mode, which is not affected by this issue. - # Running Alongside Hadoop You can run Spark and Mesos alongside your existing Hadoop cluster by just launching them as a @@ -227,11 +224,11 @@ See the [configuration page](configuration.html) for information on Spark config spark.mesos.executor.memoryOverhead executor memory * 0.10, with minimum of 384 - This value is an additive for spark.executor.memory, specified in MiB, + This value is an additive for spark.executor.memory, specified in MB, which is used to calculate the total Mesos task memory. A value of 384 - implies a 384MiB overhead. Additionally, there is a hard-coded 7% minimum + implies a 384MB overhead. Additionally, there is a hard-coded 10% minimum overhead. The final overhead will be the larger of either - `spark.mesos.executor.memoryOverhead` or 7% of `spark.executor.memory`. + `spark.mesos.executor.memoryOverhead` or 10% of `spark.executor.memory`. diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 9c363bc87e890..2cbb4c967eb81 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -21,14 +21,14 @@ The DataFrame API is available in [Scala](api/scala/index.html#org.apache.spark. All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell` or the `pyspark` shell. -## Starting Point: SQLContext +## Starting Point: `SQLContext`
The entry point into all functionality in Spark SQL is the -[SQLContext](api/scala/index.html#org.apache.spark.sql.SQLContext) class, or one of its -descendants. To create a basic SQLContext, all you need is a SparkContext. +[`SQLContext`](api/scala/index.html#org.apache.spark.sql.`SQLContext`) class, or one of its +descendants. To create a basic `SQLContext`, all you need is a SparkContext. {% highlight scala %} val sc: SparkContext // An existing SparkContext. @@ -43,8 +43,8 @@ import sqlContext.implicits._
The entry point into all functionality in Spark SQL is the -[SQLContext](api/java/index.html#org.apache.spark.sql.SQLContext) class, or one of its -descendants. To create a basic SQLContext, all you need is a SparkContext. +[`SQLContext`](api/java/index.html#org.apache.spark.sql.SQLContext) class, or one of its +descendants. To create a basic `SQLContext`, all you need is a SparkContext. {% highlight java %} JavaSparkContext sc = ...; // An existing JavaSparkContext. @@ -56,8 +56,8 @@ SQLContext sqlContext = new org.apache.spark.sql.SQLContext(sc);
The entry point into all relational functionality in Spark is the -[SQLContext](api/python/pyspark.sql.SQLContext-class.html) class, or one -of its decedents. To create a basic SQLContext, all you need is a SparkContext. +[`SQLContext`](api/python/pyspark.sql.SQLContext-class.html) class, or one +of its decedents. To create a basic `SQLContext`, all you need is a SparkContext. {% highlight python %} from pyspark.sql import SQLContext @@ -67,20 +67,20 @@ sqlContext = SQLContext(sc)
-In addition to the basic SQLContext, you can also create a HiveContext, which provides a -superset of the functionality provided by the basic SQLContext. Additional features include +In addition to the basic `SQLContext`, you can also create a `HiveContext`, which provides a +superset of the functionality provided by the basic `SQLContext`. Additional features include the ability to write queries using the more complete HiveQL parser, access to Hive UDFs, and the -ability to read data from Hive tables. To use a HiveContext, you do not need to have an -existing Hive setup, and all of the data sources available to a SQLContext are still available. -HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default -Spark build. If these dependencies are not a problem for your application then using HiveContext -is recommended for the 1.3 release of Spark. Future releases will focus on bringing SQLContext up -to feature parity with a HiveContext. +ability to read data from Hive tables. To use a `HiveContext`, you do not need to have an +existing Hive setup, and all of the data sources available to a `SQLContext` are still available. +`HiveContext` is only packaged separately to avoid including all of Hive's dependencies in the default +Spark build. If these dependencies are not a problem for your application then using `HiveContext` +is recommended for the 1.3 release of Spark. Future releases will focus on bringing `SQLContext` up +to feature parity with a `HiveContext`. The specific variant of SQL that is used to parse queries can also be selected using the `spark.sql.dialect` option. This parameter can be changed using either the `setConf` method on -a SQLContext or by using a `SET key=value` command in SQL. For a SQLContext, the only dialect -available is "sql" which uses a simple SQL parser provided by Spark SQL. In a HiveContext, the +a `SQLContext` or by using a `SET key=value` command in SQL. For a `SQLContext`, the only dialect +available is "sql" which uses a simple SQL parser provided by Spark SQL. In a `HiveContext`, the default is "hiveql", though "sql" is also available. Since the HiveQL parser is much more complete, this is recommended for most use cases. @@ -100,7 +100,7 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) val df = sqlContext.jsonFile("examples/src/main/resources/people.json") // Displays the content of the DataFrame to stdout -df.show() +df.show() {% endhighlight %}
@@ -151,10 +151,10 @@ val df = sqlContext.jsonFile("examples/src/main/resources/people.json") // Show the content of the DataFrame df.show() -// age name +// age name // null Michael -// 30 Andy -// 19 Justin +// 30 Andy +// 19 Justin // Print the schema in a tree format df.printSchema() @@ -164,20 +164,20 @@ df.printSchema() // Select only the "name" column df.select("name").show() -// name +// name // Michael -// Andy -// Justin +// Andy +// Justin // Select everybody, but increment the age by 1 -df.select("name", df("age") + 1).show() +df.select(df("name"), df("age") + 1).show() // name (age + 1) -// Michael null -// Andy 31 -// Justin 20 +// Michael null +// Andy 31 +// Justin 20 // Select people older than 21 -df.filter(df("name") > 21).show() +df.filter(df("age") > 21).show() // age name // 30 Andy @@ -201,10 +201,10 @@ DataFrame df = sqlContext.jsonFile("examples/src/main/resources/people.json"); // Show the content of the DataFrame df.show(); -// age name +// age name // null Michael -// 30 Andy -// 19 Justin +// 30 Andy +// 19 Justin // Print the schema in a tree format df.printSchema(); @@ -214,20 +214,20 @@ df.printSchema(); // Select only the "name" column df.select("name").show(); -// name +// name // Michael -// Andy -// Justin +// Andy +// Justin // Select everybody, but increment the age by 1 -df.select("name", df.col("age").plus(1)).show(); +df.select(df.col("name"), df.col("age").plus(1)).show(); // name (age + 1) -// Michael null -// Andy 31 -// Justin 20 +// Michael null +// Andy 31 +// Justin 20 // Select people older than 21 -df.filter(df("name") > 21).show(); +df.filter(df.col("age").gt(21)).show(); // age name // 30 Andy @@ -251,10 +251,10 @@ df = sqlContext.jsonFile("examples/src/main/resources/people.json") # Show the content of the DataFrame df.show() -## age name +## age name ## null Michael -## 30 Andy -## 19 Justin +## 30 Andy +## 19 Justin # Print the schema in a tree format df.printSchema() @@ -264,20 +264,20 @@ df.printSchema() # Select only the "name" column df.select("name").show() -## name +## name ## Michael -## Andy -## Justin +## Andy +## Justin # Select everybody, but increment the age by 1 -df.select("name", df.age + 1).show() +df.select(df.name, df.age + 1).show() ## name (age + 1) -## Michael null -## Andy 31 -## Justin 20 +## Michael null +## Andy 31 +## Justin 20 # Select people older than 21 -df.filter(df.name > 21).show() +df.filter(df.age > 21).show() ## age name ## 30 Andy @@ -358,7 +358,7 @@ import sqlContext.implicits._ case class Person(name: String, age: Int) // Create an RDD of Person objects and register it as a table. -val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt)) +val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt)).toDF() people.registerTempTable("people") // SQL statements can be run by using the sql methods provided by sqlContext. @@ -797,7 +797,7 @@ When working with a `HiveContext`, `DataFrames` can also be saved as persistent contents of the dataframe and create a pointer to the data in the HiveMetastore. Persistent tables will still exist even after your Spark program has restarted, as long as you maintain your connection to the same metastore. A DataFrame for a persistent table can be created by calling the `table` -method on a SQLContext with the name of the table. +method on a `SQLContext` with the name of the table. By default `saveAsTable` will create a "managed table", meaning that the location of the data will be controlled by the metastore. Managed tables will also have their data deleted automatically @@ -907,9 +907,132 @@ SELECT * FROM parquetTable
+### Partition discovery + +Table partitioning is a common optimization approach used in systems like Hive. In a partitioned +table, data are usually stored in different directories, with partitioning column values encoded in +the path of each partition directory. The Parquet data source is now able to discover and infer +partitioning information automatically. For exmaple, we can store all our previously used +population data into a partitioned table using the following directory structure, with two extra +columns, `gender` and `country` as partitioning columns: + +{% highlight text %} + +path +└── to + └── table + ├── gender=male + │   ├── ... + │   │ + │   ├── country=US + │   │   └── data.parquet + │   ├── country=CN + │   │   └── data.parquet + │   └── ... + └── gender=female +    ├── ... +    │ +    ├── country=US +    │   └── data.parquet +    ├── country=CN +    │   └── data.parquet +    └── ... + +{% endhighlight %} + +By passing `path/to/table` to either `SQLContext.parquetFile` or `SQLContext.load`, Spark SQL will +automatically extract the partitioning information from the paths. Now the schema of the returned +DataFrame becomes: + +{% highlight text %} + +root +|-- name: string (nullable = true) +|-- age: long (nullable = true) +|-- gender: string (nullable = true) +|-- country: string (nullable = true) + +{% endhighlight %} + +Notice that the data types of the partitioning columns are automatically inferred. Currently, +numeric data types and string type are supported. + +### Schema merging + +Like ProtocolBuffer, Avro, and Thrift, Parquet also supports schema evolution. Users can start with +a simple schema, and gradually add more columns to the schema as needed. In this way, users may end +up with multiple Parquet files with different but mutually compatible schemas. The Parquet data +source is now able to automatically detect this case and merge schemas of all these files. + +
+ +
+ +{% highlight scala %} +// sqlContext from the previous example is used in this example. +// This is used to implicitly convert an RDD to a DataFrame. +import sqlContext.implicits._ + +// Create a simple DataFrame, stored into a partition directory +val df1 = sparkContext.makeRDD(1 to 5).map(i => (i, i * 2)).toDF("single", "double") +df1.saveAsParquetFile("data/test_table/key=1") + +// Create another DataFrame in a new partition directory, +// adding a new column and dropping an existing column +val df2 = sparkContext.makeRDD(6 to 10).map(i => (i, i * 3)).toDF("single", "triple") +df2.saveAsParquetFile("data/test_table/key=2") + +// Read the partitioned table +val df3 = sqlContext.parquetFile("data/test_table") +df3.printSchema() + +// The final schema consists of all 3 columns in the Parquet files together +// with the partiioning column appeared in the partition directory paths. +// root +// |-- single: int (nullable = true) +// |-- double: int (nullable = true) +// |-- triple: int (nullable = true) +// |-- key : int (nullable = true) +{% endhighlight %} + +
+ +
+ +{% highlight python %} +# sqlContext from the previous example is used in this example. + +# Create a simple DataFrame, stored into a partition directory +df1 = sqlContext.createDataFrame(sc.parallelize(range(1, 6))\ + .map(lambda i: Row(single=i, double=i * 2))) +df1.save("data/test_table/key=1", "parquet") + +# Create another DataFrame in a new partition directory, +# adding a new column and dropping an existing column +df2 = sqlContext.createDataFrame(sc.parallelize(range(6, 11)) + .map(lambda i: Row(single=i, triple=i * 3))) +df2.save("data/test_table/key=2", "parquet") + +# Read the partitioned table +df3 = sqlContext.parquetFile("data/test_table") +df3.printSchema() + +# The final schema consists of all 3 columns in the Parquet files together +# with the partiioning column appeared in the partition directory paths. +# root +# |-- single: int (nullable = true) +# |-- double: int (nullable = true) +# |-- triple: int (nullable = true) +# |-- key : int (nullable = true) +{% endhighlight %} + +
+ +
+ ### Configuration -Configuration of Parquet can be done using the `setConf` method on SQLContext or by running +Configuration of Parquet can be done using the `setConf` method on `SQLContext` or by running `SET key=value` commands using SQL. @@ -972,7 +1095,7 @@ Configuration of Parquet can be done using the `setConf` method on SQLContext or
Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using one of two methods in a SQLContext: +This conversion can be done using one of two methods in a `SQLContext`: * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. * `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. @@ -1014,7 +1137,7 @@ val anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD)
Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using one of two methods in a SQLContext : +This conversion can be done using one of two methods in a `SQLContext` : * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. * `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. @@ -1056,7 +1179,7 @@ DataFrame anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD);
Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using one of two methods in a SQLContext: +This conversion can be done using one of two methods in a `SQLContext`: * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. * `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. @@ -1085,7 +1208,7 @@ people.printSchema() # Register this DataFrame as a table. people.registerTempTable("people") -# SQL statements can be run by using the sql methods provided by sqlContext. +# SQL statements can be run by using the sql methods provided by `sqlContext`. teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") # Alternatively, a DataFrame can be created for a JSON dataset represented by @@ -1131,7 +1254,7 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and adds support for finding tables in the MetaStore and writing queries using HiveQL. Users who do -not have an existing Hive deployment can still create a HiveContext. When not configured by the +not have an existing Hive deployment can still create a `HiveContext`. When not configured by the hive-site.xml, the context automatically creates `metastore_db` and `warehouse` in the current directory. @@ -1318,7 +1441,7 @@ Spark SQL can cache tables using an in-memory columnar format by calling `sqlCon Then Spark SQL will scan only required columns and will automatically tune compression to minimize memory usage and GC pressure. You can call `sqlContext.uncacheTable("tableName")` to remove the table from memory. -Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running +Configuration of in-memory caching can be done using the `setConf` method on `SQLContext` or by running `SET key=value` commands using SQL.
@@ -1429,10 +1552,10 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. You may also use the beeline script that comes with Hive. -Thrift JDBC server also supports sending thrift RPC messages over HTTP transport. -Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`: +Thrift JDBC server also supports sending thrift RPC messages over HTTP transport. +Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`: - hive.server2.transport.mode - Set this to value: http + hive.server2.transport.mode - Set this to value: http hive.server2.thrift.http.port - HTTP port number fo listen on; default is 10001 hive.server2.http.endpoint - HTTP endpoint; default is cliservice @@ -1506,7 +1629,7 @@ When using function inside of the DSL (now replaced with the `DataFrame` API) us Spark 1.3 removes the type aliases that were present in the base sql package for `DataType`. Users should instead import the classes in `org.apache.spark.sql.types` -#### UDF Registration Moved to sqlContext.udf (Java & Scala) +#### UDF Registration Moved to `sqlContext.udf` (Java & Scala) Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been moved into the udf object in `SQLContext`. diff --git a/docs/streaming-flume-integration.md b/docs/streaming-flume-integration.md index 40e17246fea83..c8ab146bcae0a 100644 --- a/docs/streaming-flume-integration.md +++ b/docs/streaming-flume-integration.md @@ -5,6 +5,8 @@ title: Spark Streaming + Flume Integration Guide [Apache Flume](https://flume.apache.org/) is a distributed, reliable, and available service for efficiently collecting, aggregating, and moving large amounts of log data. Here we explain how to configure Flume and Spark Streaming to receive data from Flume. There are two approaches to this. +Python API Flume is not yet available in the Python API. + ## Approach 1: Flume-style Push-based Approach Flume is designed to push data between Flume agents. In this approach, Spark Streaming essentially sets up a receiver that acts an Avro agent for Flume, to which Flume can push the data. Here are the configuration steps. diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md index 77c0abbbacbd0..64714f0b799fc 100644 --- a/docs/streaming-kafka-integration.md +++ b/docs/streaming-kafka-integration.md @@ -2,58 +2,155 @@ layout: global title: Spark Streaming + Kafka Integration Guide --- -[Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Here we explain how to configure Spark Streaming to receive data from Kafka. +[Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Here we explain how to configure Spark Streaming to receive data from Kafka. There are two approaches to this - the old approach using Receivers and Kafka's high-level API, and a new experimental approach (introduced in Spark 1.3) without using Receivers. They have different programming models, performance characteristics, and semantics guarantees, so read on for more details. -1. **Linking:** In your SBT/Maven project definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). +## Approach 1: Receiver-based Approach +This approach uses a Receiver to receive the data. The Received is implemented using the Kafka high-level consumer API. As with all receivers, the data received from Kafka through a Receiver is stored in Spark executors, and then jobs launched by Spark Streaming processes the data. + +However, under default configuration, this approach can lose data under failures (see [receiver reliability](streaming-programming-guide.html#receiver-reliability). To ensure zero-data loss, you have to additionally enable Write Ahead Logs in Spark Streaming. To ensure zero data loss, enable the Write Ahead Logs (introduced in Spark 1.2). This synchronously saves all the received Kafka data into write ahead logs on a distributed file system (e.g HDFS), so that all the data can be recovered on failure. See [Deploying section](streaming-programming-guide.html#deploying-applications) in the streaming programming guide for more details on Write Ahead Logs. + +Next, we discuss how to use this approach in your streaming application. + +1. **Linking:** For Scala/Java applications using SBT/Maven project definitions, link your streaming application with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). groupId = org.apache.spark artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}} version = {{site.SPARK_VERSION_SHORT}} -2. **Programming:** In the streaming application code, import `KafkaUtils` and create input DStream as follows. + For Python applications, you will have to add this above library and its dependencies when deploying your application. See the *Deploying* subsection below. + +2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows.
import org.apache.spark.streaming.kafka._ - val kafkaStream = KafkaUtils.createStream( - streamingContext, [zookeeperQuorum], [group id of the consumer], [per-topic number of Kafka partitions to consume]) + val kafkaStream = KafkaUtils.createStream(streamingContext, + [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) - See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) + You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala).
import org.apache.spark.streaming.kafka.*; - JavaPairReceiverInputDStream kafkaStream = KafkaUtils.createStream( - streamingContext, [zookeeperQuorum], [group id of the consumer], [per-topic number of Kafka partitions to consume]); + JavaPairReceiverInputDStream kafkaStream = + KafkaUtils.createStream(streamingContext, + [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]); - See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html) + You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html) and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java). + +
+
+ from pyspark.streaming.kafka import KafkaUtils + + kafkaStream = KafkaUtils.createStream(streamingContext, \ + [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) + + By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils) + and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/kafka_wordcount.py).
- *Points to remember:* + **Points to remember:** - Topic partitions in Kafka does not correlate to partitions of RDDs generated in Spark Streaming. So increasing the number of topic-specific partitions in the `KafkaUtils.createStream()` only increases the number of threads using which topics that are consumed within a single receiver. It does not increase the parallelism of Spark in processing the data. Refer to the main document for more information on that. - Multiple Kafka input DStreams can be created with different groups and topics for parallel receiving of data using multiple receivers. -3. **Deploying:** Package `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). - -Note that the Kafka receiver used by default is an -[*unreliable* receiver](streaming-programming-guide.html#receiver-reliability) section in the -programming guide). In Spark 1.2, we have added an experimental *reliable* Kafka receiver that -provides stronger -[fault-tolerance guarantees](streaming-programming-guide.html#fault-tolerance-semantics) of zero -data loss on failures. This receiver is automatically used when the write ahead log -(also introduced in Spark 1.2) is enabled -(see [Deployment](#deploying-applications.html) section in the programming guide). This -may reduce the receiving throughput of individual Kafka receivers compared to the unreliable -receivers, but this can be corrected by running -[more receivers in parallel](streaming-programming-guide.html#level-of-parallelism-in-data-receiving) -to increase aggregate throughput. Additionally, it is recommended that the replication of the -received data within Spark be disabled when the write ahead log is enabled as the log is already stored -in a replicated storage system. This can be done by setting the storage level for the input -stream to `StorageLevel.MEMORY_AND_DISK_SER` (that is, use + - If you have enabled Write Ahead Logs with a replicated file system like HDFS, the received data is already being replicated in the log. Hence, the storage level in storage level for the input stream to `StorageLevel.MEMORY_AND_DISK_SER` (that is, use `KafkaUtils.createStream(..., StorageLevel.MEMORY_AND_DISK_SER)`). + +3. **Deploying:** As with any Spark applications, `spark-submit` is used to launch your application. However, the details are slightly different for Scala/Java applications and Python applications. + + For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + + For Python applications which lack SBT/Maven project management, `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies can be directly added to `spark-submit` using `--packages` (see [Application Submission Guide](submitting-applications.html)). That is, + + ./bin/spark-submit --packages org.apache.spark:spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... + + Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-kafka-assembly` from the + [Maven repository](http://search.maven.org/#search|ga|1|a%3A%22spark-streaming-kafka-assembly_2.10%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`. + +## Approach 2: Direct Approach (No Receivers) +This is a new receiver-less "direct" approach has been introduced in Spark 1.3 to ensure stronger end-to-end guarantees. Instead of using receivers to receive data, this approach periodically queries Kafka for the latest offsets in each topic+partition, and accordingly defines the offset ranges to process in each batch. When the jobs to process the data are launched, Kafka's simple consumer API is used to read the defined ranges of offsets from Kafka (similar to read files from a file system). Note that this is an experimental feature in Spark 1.3 and is only available in the Scala and Java API. + +This approach has the following advantages over the received-based approach (i.e. Approach 1). + +- *Simplified Parallelism:* No need to create multiple input Kafka streams and union-ing them. With `directStream`, Spark Streaming will create as many RDD partitions as there is Kafka partitions to consume, which will all read data from Kafka in parallel. So there is one-to-one mapping between Kafka and RDD partitions, which is easier to understand and tune. + +- *Efficiency:* Achieving zero-data loss in the first approach required the data to be stored in a Write Ahead Log, which further replicated the data. This is actually inefficient as the data effectively gets replicated twice - once by Kafka, and a second time by the Write Ahead Log. This second approach eliminate the problem as there is no receiver, and hence no need for Write Ahead Logs. + +- *Exactly-once semantics:* The first approach uses Kafka's high level API to store consumed offsets in Zookeeper. This is traditionally the way to consume data from Kafka. While this approach (in combination with write ahead logs) can ensure zero data loss (i.e. at-least once semantics), there is a small chance some records may get consumed twice under some failures. This occurs because of inconsistencies between data reliably received by Spark Streaming and offsets tracked by Zookeeper. Hence, in this second approach, we use simple Kafka API that does not use Zookeeper and offsets tracked only by Spark Streaming within its checkpoints. This eliminates inconsistencies between Spark Streaming and Zookeeper/Kafka, and so each record is received by Spark Streaming effectively exactly once despite failures. + +Note that one disadvantage of this approach is that it does not update offsets in Zookeeper, hence Zookeeper-based Kafka monitoring tools will not show progress. However, you can access the offsets processed by this approach in each batch and update Zookeeper yourself (see below). + +Next, we discuss how to use this approach in your streaming application. + +1. **Linking:** This approach is supported only in Scala/Java application. Link your SBT/Maven project with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + +2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows. + +
+
+ import org.apache.spark.streaming.kafka._ + + val directKafkaStream = KafkaUtils.createDirectStream[ + [key class], [value class], [key decoder class], [value decoder class] ]( + streamingContext, [map of Kafka parameters], [set of topics to consume]) + + See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) + and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala). +
+
+ import org.apache.spark.streaming.kafka.*; + + JavaPairReceiverInputDStream directKafkaStream = + KafkaUtils.createDirectStream(streamingContext, + [key class], [value class], [key decoder class], [value decoder class], + [map of Kafka parameters], [set of topics to consume]); + + See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html) + and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java). + +
+
+ + In the Kafka parameters, you must specify either `metadata.broker.list` or `bootstrap.servers`. + By default, it will start consuming from the latest offset of each Kafka partition. If you set configuration `auto.offset.reset` in Kafka parameters to `smallest`, then it will start consuming from the smallest offset. + + You can also start consuming from any arbitrary offset using other variations of `KafkaUtils.createDirectStream`. Furthermore, if you want to access the Kafka offsets consumed in each batch, you can do the following. + +
+
+ directKafkaStream.foreachRDD { rdd => + val offsetRanges = rdd.asInstanceOf[HasOffsetRanges] + // offsetRanges.length = # of Kafka partitions being consumed + ... + } +
+
+ directKafkaStream.foreachRDD( + new Function, Void>() { + @Override + public Void call(JavaPairRDD rdd) throws IOException { + OffsetRange[] offsetRanges = ((HasOffsetRanges)rdd).offsetRanges + // offsetRanges.length = # of Kafka partitions being consumed + ... + return null; + } + } + ); +
+
+ + You can use this to update Zookeeper yourself if you want Zookeeper-based Kafka monitoring tools to show progress of the streaming application. + + Another thing to note is that since this approach does not use Receivers, the standard receiver-related (that is, [configurations](configuration.html) of the form `spark.streaming.receiver.*` ) will not apply to the input DStreams created by this approach (will apply to other input DStreams though). Instead, use the [configurations](configuration.html) `spark.streaming.kafka.*`. An important one is `spark.streaming.kafka.maxRatePerPartition` which is the maximum rate at which each Kafka partition will be read by this direct API. + +3. **Deploying:** Similar to the first approach, you can package `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR and the launch the application using `spark-submit`. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. \ No newline at end of file diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 815c98713b738..6d6229625f3f9 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -189,15 +189,15 @@ Next, we want to count these words. {% highlight java %} // Count each word in each batch -JavaPairDStream pairs = words.map( +JavaPairDStream pairs = words.mapToPair( new PairFunction() { - @Override public Tuple2 call(String s) throws Exception { + @Override public Tuple2 call(String s) { return new Tuple2(s, 1); } }); JavaPairDStream wordCounts = pairs.reduceByKey( new Function2() { - @Override public Integer call(Integer i1, Integer i2) throws Exception { + @Override public Integer call(Integer i1, Integer i2) { return i1 + i2; } }); @@ -432,7 +432,7 @@ some of the common ones are as follows.
For an up-to-date list, please refer to the -[Apache repository](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) +[Maven repository](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) for the full list of supported sources and artifacts. *** @@ -662,8 +662,7 @@ methods for creating DStreams from files and Akka actors as input sources. For simple text files, there is an easier method `streamingContext.textFileStream(dataDirectory)`. And file streams do not require running a receiver, hence does not require allocating cores. - Python API As of Spark 1.2, - `fileStream` is not available in the Python API, only `textFileStream` is available. + Python API `fileStream` is not available in the Python API, only `textFileStream` is available. - **Streams based on Custom Actors:** DStreams can be created with data streams received through Akka actors by using `streamingContext.actorStream(actorProps, actor-name)`. See the [Custom Receiver @@ -682,8 +681,9 @@ for Java, and [StreamingContext](api/python/pyspark.streaming.html#pyspark.strea ### Advanced Sources {:.no_toc} -Python API As of Spark 1.2, -these sources are not available in the Python API. + +Python API As of Spark 1.3, +out of these sources, *only* Kafka is available in the Python API. We will add more advanced sources in the Python API in future. This category of sources require interfacing with external non-Spark libraries, some of them with complex dependencies (e.g., Kafka and Flume). Hence, to minimize issues related to version conflicts @@ -723,6 +723,12 @@ and it in the classpath. Some of these advanced sources are as follows. +- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka 0.8.1.1. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details. + +- **Flume:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Flume 1.4.0. See the [Flume Integration Guide](streaming-flume-integration.html) for more details. + +- **Kinesis:** See the [Kinesis Integration Guide](streaming-kinesis-integration.html) for more details. + - **Twitter:** Spark Streaming's TwitterUtils uses Twitter4j 3.0.3 to get the public stream of tweets using [Twitter's Streaming API](https://dev.twitter.com/docs/streaming-apis). Authentication information can be provided by any of the [methods](http://twitter4j.org/en/configuration.html) supported by @@ -732,17 +738,10 @@ Some of these advanced sources are as follows. ([TwitterPopularTags]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala) and [TwitterAlgebirdCMS]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala)). -- **Flume:** Spark Streaming {{site.SPARK_VERSION_SHORT}} can received data from Flume 1.4.0. See the [Flume Integration Guide](streaming-flume-integration.html) for more details. - -- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} can receive data from Kafka 0.8.0. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details. - -- **Kinesis:** See the [Kinesis Integration Guide](streaming-kinesis-integration.html) for more details. - ### Custom Sources {:.no_toc} -Python API As of Spark 1.2, -these sources are not available in the Python API. +Python API This is not yet supported in Python. Input DStreams can also be created out of custom data sources. All you have to do is implement an user-defined **receiver** (see next section to understand what that is) that can receive data from @@ -846,7 +845,7 @@ Some of the common ones are as follows. -The last two transformations are worth highlighting again. +A few of these transformations are worth discussing in more detail. #### UpdateStateByKey Operation {:.no_toc} @@ -997,7 +996,7 @@ In fact, you can also use [machine learning](mllib-guide.html) and #### Window Operations {:.no_toc} -Finally, Spark Streaming also provides *windowed computations*, which allow you to apply +Spark Streaming also provides *windowed computations*, which allow you to apply transformations over a sliding window of data. This following figure illustrates this sliding window. @@ -1041,7 +1040,7 @@ val windowedWordCounts = pairs.reduceByKeyAndWindow((a:Int,b:Int) => (a + b), Se {% highlight java %} // Reduce function adding two integers, defined separately for clarity Function2 reduceFunc = new Function2() { - @Override public Integer call(Integer i1, Integer i2) throws Exception { + @Override public Integer call(Integer i1, Integer i2) { return i1 + i2; } }; @@ -1120,6 +1119,100 @@ said two parameters - windowLength and slideInterval. +#### Join Operations +{:.no_toc} +Finally, its worth highlighting how easily you can perform different kinds of joins in Spark Streaming. + + +##### Stream-stream joins +{:.no_toc} +Streams can be very easily joined with other streams. + +
+
+{% highlight scala %} +val stream1: DStream[String, String] = ... +val stream2: DStream[String, String] = ... +val joinedStream = stream1.join(stream2) +{% endhighlight %} +
+
+{% highlight java %} +JavaPairDStream stream1 = ... +JavaPairDStream stream2 = ... +JavaPairDStream joinedStream = stream1.join(stream2); +{% endhighlight %} +
+
+{% highlight python %} +stream1 = ... +stream2 = ... +joinedStream = stream1.join(stream2) +{% endhighlight %} +
+
+Here, in each batch interval, the RDD generated by `stream1` will be joined with the RDD generated by `stream2`. You can also do `leftOuterJoin`, `rightOuterJoin`, `fullOuterJoin`. Furthermore, it is often very useful to do joins over windows of the streams. That is pretty easy as well. + +
+
+{% highlight scala %} +val windowedStream1 = stream1.window(Seconds(20)) +val windowedStream2 = stream2.window(Minutes(1)) +val joinedStream = windowedStream1.join(windowedStream2) +{% endhighlight %} +
+
+{% highlight java %} +JavaPairDStream windowedStream1 = stream1.window(Durations.seconds(20)); +JavaPairDStream windowedStream2 = stream2.window(Durations.minutes(1)); +JavaPairDStream joinedStream = windowedStream1.join(windowedStream2); +{% endhighlight %} +
+
+{% highlight python %} +windowedStream1 = stream1.window(20) +windowedStream2 = stream2.window(60) +joinedStream = windowedStream1.join(windowedStream2) +{% endhighlight %} +
+
+ +##### Stream-dataset joins +{:.no_toc} +This has already been shown earlier while explain `DStream.transform` operation. Here is yet another example of joining a windowed stream with a dataset. + +
+
+{% highlight scala %} +val dataset: RDD[String, String] = ... +val windowedStream = stream.window(Seconds(20))... +val joinedStream = windowedStream.transform { rdd => rdd.join(dataset) } +{% endhighlight %} +
+
+{% highlight java %} +JavaPairRDD dataset = ... +JavaPairDStream windowedStream = stream.window(Durations.seconds(20)); +JavaPairDStream joinedStream = windowedStream.transform( + new Function>, JavaRDD>>() { + @Override + public JavaRDD> call(JavaRDD> rdd) { + return rdd.join(dataset); + } + } +); +{% endhighlight %} +
+
+{% highlight python %} +dataset = ... # some RDD +windowedStream = stream.window(20) +joinedStream = windowedStream.transform(lambda rdd: rdd.join(dataset)) +{% endhighlight %} +
+
+ +In fact, you can also dynamically change the dataset you want to join against. The function provided to `transform` is evaluated every batch interval and therefore will use the current dataset that `dataset` reference points to. The complete list of DStream transformations is available in the API documentation. For the Scala API, see [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) @@ -1327,6 +1420,178 @@ Note that the connections in the pool should be lazily created on demand and tim *** +## DataFrame and SQL Operations +You can easily use [DataFrames and SQL](sql-programming-guide.html) operations on streaming data. You have to create a SQLContext using the SparkContext that the StreamingContext is using. Furthermore this has to done such that it can be restarted on driver failures. This is done by creating a lazily instantiated singleton instance of SQLContext. This is shown in the following example. It modifies the earlier [word count example](#a-quick-example) to generate word counts using DataFrames and SQL. Each RDD is converted to a DataFrame, registered as a temporary table and then queried using SQL. + +
+
+{% highlight scala %} + +/** Lazily instantiated singleton instance of SQLContext */ +object SQLContextSingleton { + @transient private var instance: SQLContext = null + + // Instantiate SQLContext on demand + def getInstance(sparkContext: SparkContext): SQLContext = synchronized { + if (instance == null) { + instance = new SQLContext(sparkContext) + } + instance + } +} + +... + +/** Case class for converting RDD to DataFrame */ +case class Row(word: String) + +... + +/** DataFrame operations inside your streaming program */ + +val words: DStream[String] = ... + +words.foreachRDD { rdd => + + // Get the singleton instance of SQLContext + val sqlContext = SQLContextSingleton.getInstance(rdd.sparkContext) + import sqlContext.implicits._ + + // Convert RDD[String] to RDD[case class] to DataFrame + val wordsDataFrame = rdd.map(w => Row(w)).toDF() + + // Register as table + wordsDataFrame.registerTempTable("words") + + // Do word count on DataFrame using SQL and print it + val wordCountsDataFrame = + sqlContext.sql("select word, count(*) as total from words group by word") + wordCountsDataFrame.show() +} + +{% endhighlight %} + +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala). +
+
+{% highlight java %} + +/** Lazily instantiated singleton instance of SQLContext */ +class JavaSQLContextSingleton { + static private transient SQLContext instance = null; + static public SQLContext getInstance(SparkContext sparkContext) { + if (instance == null) { + instance = new SQLContext(sparkContext); + } + return instance; + } +} + +... + +/** Java Bean class for converting RDD to DataFrame */ +public class JavaRow implements java.io.Serializable { + private String word; + + public String getWord() { + return word; + } + + public void setWord(String word) { + this.word = word; + } +} + +... + +/** DataFrame operations inside your streaming program */ + +JavaDStream words = ... + +words.foreachRDD( + new Function2, Time, Void>() { + @Override + public Void call(JavaRDD rdd, Time time) { + SQLContext sqlContext = JavaSQLContextSingleton.getInstance(rdd.context()); + + // Convert RDD[String] to RDD[case class] to DataFrame + JavaRDD rowRDD = rdd.map(new Function() { + public JavaRow call(String word) { + JavaRow record = new JavaRow(); + record.setWord(word); + return record; + } + }); + DataFrame wordsDataFrame = sqlContext.createDataFrame(rowRDD, JavaRow.class); + + // Register as table + wordsDataFrame.registerTempTable("words"); + + // Do word count on table using SQL and print it + DataFrame wordCountsDataFrame = + sqlContext.sql("select word, count(*) as total from words group by word"); + wordCountsDataFrame.show(); + return null; + } + } +); +{% endhighlight %} + +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java). +
+
+{% highlight python %} + +# Lazily instantiated global instance of SQLContext +def getSqlContextInstance(sparkContext): + if ('sqlContextSingletonInstance' not in globals()): + globals()['sqlContextSingletonInstance'] = SQLContext(sparkContext) + return globals()['sqlContextSingletonInstance'] + +... + +# DataFrame operations inside your streaming program + +words = ... # DStream of strings + +def process(time, rdd): + print "========= %s =========" % str(time) + try: + # Get the singleton instance of SQLContext + sqlContext = getSqlContextInstance(rdd.context) + + # Convert RDD[String] to RDD[Row] to DataFrame + rowRdd = rdd.map(lambda w: Row(word=w)) + wordsDataFrame = sqlContext.createDataFrame(rowRdd) + + # Register as table + wordsDataFrame.registerTempTable("words") + + # Do word count on table using SQL and print it + wordCountsDataFrame = sqlContext.sql("select word, count(*) as total from words group by word") + wordCountsDataFrame.show() + except: + pass + +words.foreachRDD(process) +{% endhighlight %} + +See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/sql_network_wordcount.py). + +
+
+ +You can also run SQL queries on tables defined on streaming data from a different thread (that is, asynchronous to the running StreamingContext). Just make sure that you set the StreamingContext to remember sufficient amount of streaming data such that query can run. Otherwise the StreamingContext, which is unaware of the any asynchronous SQL queries, will delete off old streaming data before the query can complete. For example, if you want to query the last batch, but your query can take 5 minutes to run, then call `streamingContext.remember(Minutes(5))` (in Scala, or equivalent in other languages). + +See the [DataFrames and SQL](sql-programming-guide.html) guide to learn more about DataFrames. + +*** + +## MLlib Operations +You can also easily use machine learning algorithms provided by [MLlib](mllib-guide.html). First of all, there are streaming machine learning algorithms (e.g. (Streaming Linear Regression](mllib-linear-methods.html#streaming-linear-regression), [Streaming KMeans](mllib-clustering.html#streaming-k-means), etc.) which can simultaneously learn from the streaming data as well as apply the model on the streaming data. Beyond these, for a much larger class of machine learning algorithms, you can learn a learning model offline (i.e. using historical data) and then apply the model online on streaming data. See the [MLlib](mllib-guide.html) guide for more details. + +*** + ## Caching / Persistence Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream will automatically persist every RDD of that DStream in @@ -1580,9 +1845,8 @@ To run a Spark Streaming applications, you need to have the following. + *Mesos* - [Marathon](https://github.com/mesosphere/marathon) has been used to achieve this with Mesos. - -- *[Experimental in Spark 1.2] Configuring write ahead logs* - In Spark 1.2, - we have introduced a new experimental feature of write ahead logs for achieving strong +- *[Since Spark 1.2] Configuring write ahead logs* - Since Spark 1.2, + we have introduced _write ahead logs_ for achieving strong fault-tolerance guarantees. If enabled, all the data received from a receiver gets written into a write ahead log in the configuration checkpoint directory. This prevents data loss on driver recovery, thus ensuring zero data loss (discussed in detail in the @@ -1668,7 +1932,7 @@ improve the performance of you application. At a high level, you need to conside 2. Setting the right batch size such that the batches of data can be processed as fast as they are received (that is, data processing keeps up with the data ingestion). -## Reducing the Processing Time of each Batch +## Reducing the Batch Processing Times There are a number of optimizations that can be done in Spark to minimize the processing time of each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section highlights some of the most important ones. @@ -1740,16 +2004,15 @@ documentation), or set the `spark.default.parallelism` ### Data Serialization {:.no_toc} -The overhead of data serialization can be significant, especially when sub-second batch sizes are - to be achieved. There are two aspects to it. +The overheads of data serialization can be reduce by tuning the serialization formats. In case of streaming, there are two types of data that are being serialized. + +* **Input data**: By default, the input data received through Receivers is stored in the executors' memory with [StorageLevel.MEMORY_AND_DISK_SER_2](api/scala/index.html#org.apache.spark.storage.StorageLevel$). That is, the data is serialized into bytes to reduce GC overheads, and replicated for tolerating executor failures. Also, the data is kept first in memory, and spilled over to disk only if the memory is unsufficient to hold all the input data necessary for the streaming computation. This serialization obviously has overheads -- the receiver must deserialize the received data and re-serialize it using Spark's serialization format. -* **Serialization of RDD data in Spark**: Please refer to the detailed discussion on data - serialization in the [Tuning Guide](tuning.html). However, note that unlike Spark, by default - RDDs are persisted as serialized byte arrays to minimize pauses related to GC. +* **Persisted RDDs generated by Streaming Operations**: RDDs generated by streaming computations may be persisted in memory. For example, window operation persist data in memory as they would be processed multiple times. However, unlike Spark, by default RDDs are persisted with [StorageLevel.MEMORY_ONLY_SER](api/scala/index.html#org.apache.spark.storage.StorageLevel$) (i.e. serialized) to minimize GC overheads. -* **Serialization of input data**: To ingest external data into Spark, data received as bytes - (say, from the network) needs to deserialized from bytes and re-serialized into Spark's - serialization format. Hence, the deserialization overhead of input data may be a bottleneck. +In both cases, using Kryo serialization can reduce both CPU and memory overheads. See the [Spark Tuning Guide](tuning.html#data-serialization)) for more details. Consider registering custom classes, and disabling object reference tracking for Kryo (see Kryo-related configurations in the [Configuration Guide](configuration.html#compression-and-serialization)). + +In specific cases where the amount of data that needs to be retained for the streaming application is not large, it may be feasible to persist data (both types) as deserialized objects without incurring excessive GC overheads. For example, if you are using batch intervals of few seconds and no window operations, then you can try disabling serialization in persisted data by explicitly setting the storage level accordingly. This would reduce the CPU overheads due to serialization, potentially improving performance without too much GC overheads. ### Task Launching Overheads {:.no_toc} @@ -1769,7 +2032,7 @@ thus allowing sub-second batch size to be viable. *** -## Setting the Right Batch Size +## Setting the Right Batch Interval For a Spark Streaming application running on a cluster to be stable, the system should be able to process data as fast as it is being received. In other words, batches of data should be processed as fast as they are being generated. Whether this is true for an application can be found by @@ -1801,40 +2064,40 @@ temporary data rate increases maybe fine as long as the delay reduces back to a ## Memory Tuning Tuning the memory usage and GC behavior of Spark applications have been discussed in great detail -in the [Tuning Guide](tuning.html). It is recommended that you read that. In this section, -we highlight a few customizations that are strongly recommended to minimize GC related pauses -in Spark Streaming applications and achieving more consistent batch processing times. - -* **Default persistence level of DStreams**: Unlike RDDs, the default persistence level of DStreams -serializes the data in memory (that is, -[StorageLevel.MEMORY_ONLY_SER](api/scala/index.html#org.apache.spark.storage.StorageLevel$) for -DStream compared to -[StorageLevel.MEMORY_ONLY](api/scala/index.html#org.apache.spark.storage.StorageLevel$) for RDDs). -Even though keeping the data serialized incurs higher serialization/deserialization overheads, -it significantly reduces GC pauses. - -* **Clearing persistent RDDs**: By default, all persistent RDDs generated by Spark Streaming will - be cleared from memory based on Spark's built-in policy (LRU). If `spark.cleaner.ttl` is set, - then persistent RDDs that are older than that value are periodically cleared. As mentioned - [earlier](#operation), this needs to be careful set based on operations used in the Spark - Streaming program. However, a smarter unpersisting of RDDs can be enabled by setting the - [configuration property](configuration.html#spark-properties) `spark.streaming.unpersist` to - `true`. This makes the system to figure out which RDDs are not necessary to be kept around and - unpersists them. This is likely to reduce - the RDD memory usage of Spark, potentially improving GC behavior as well. - -* **Concurrent garbage collector**: Using the concurrent mark-and-sweep GC further -minimizes the variability of GC pauses. Even though concurrent GC is known to reduce the +in the [Tuning Guide](tuning.html#memory-tuning). It is strongly recommended that you read that. In this section, we discuss a few tuning parameters specifically in the context of Spark Streaming applications. + +The amount of cluster memory required by a Spark Streaming application depends heavily on the type of transformations used. For example, if you want to use a window operation on last 10 minutes of data, then your cluster should have sufficient memory to hold 10 minutes of worth of data in memory. Or if you want to use `updateStateByKey` with a large number of keys, then the necessary memory will be high. On the contrary, if you want to do a simple map-filter-store operation, then necessary memory will be low. + +In general, since the data received through receivers are stored with StorageLevel.MEMORY_AND_DISK_SER_2, the data that does not fit in memory will spill over to the disk. This may reduce the performance of the streaming application, and hence it is advised to provide sufficient memory as required by your streaming application. Its best to try and see the memory usage on a small scale and estimate accordingly. + +Another aspect of memory tuning is garbage collection. For a streaming application that require low latency, it is undesirable to have large pauses caused by JVM Garbage Collection. + +There are a few parameters that can help you tune the memory usage and GC overheads. + +* **Persistence Level of DStreams**: As mentioned earlier in the [Data Serialization](#data-serialization) section, the input data and RDDs are by default persisted as serialized bytes. This reduces both, the memory usage and GC overheads, compared to deserialized persistence. Enabling Kryo serialization further reduces serialized sizes and memory usage. Further reduction in memory usage can be achieved with compression (see the Spark configuration `spark.rdd.compress`), at the cost of CPU time. + +* **Clearing old data**: By default, all input data and persisted RDDs generated by DStream transformations are automatically cleared. Spark Streaming decides when to clear the data based on the transformations that are used. For example, if you are using window operation of 10 minutes, then Spark Streaming will keep around last 10 minutes of data, and actively throw away older data. +Data can be retained for longer duration (e.g. interactively querying older data) by setting `streamingContext.remember`. + +* **CMS Garbage Collector**: Use of the concurrent mark-and-sweep GC is strongly recommended for keeping GC-related pauses consistently low. Even though concurrent GC is known to reduce the overall processing throughput of the system, its use is still recommended to achieve more -consistent batch processing times. +consistent batch processing times. Make sure you set the CMS GC on both the driver (using `--driver-java-options` in `spark-submit`) and the executors (using [Spark configuration](configuration.html#runtime-environment) `spark.executor.extraJavaOptions`). + +* **Other tips**: To further reduce GC overheads, here are some more tips to try. + - Use Tachyon for off-heap storage of persisted RDDs. See more detail in the [Spark Programming Guide](programming-guide.html#rdd-persistence). + - Use more executors with smaller heap sizes. This will reduce the GC pressure within each JVM heap. + *************************************************************************************************** *************************************************************************************************** # Fault-tolerance Semantics In this section, we will discuss the behavior of Spark Streaming applications in the event -of node failures. To understand this, let us remember the basic fault-tolerance semantics of -Spark's RDDs. +of failures. + +## Background +{:.no_toc} +To understand the semantics provided by Spark Streaming, let us remember the basic fault-tolerance semantics of Spark's RDDs. 1. An RDD is an immutable, deterministically re-computable, distributed dataset. Each RDD remembers the lineage of deterministic operations that were used on a fault-tolerant input @@ -1868,13 +2131,43 @@ Furthermore, there are two kinds of failures that we should be concerned about: With this basic knowledge, let us understand the fault-tolerance semantics of Spark Streaming. -## Semantics with files as input source +## Definitions +{:.no_toc} +The semantics of streaming systems are often captured in terms of how many times each record can be processed by the system. There are three types of guarantees that a system can provide under all possible operating conditions (despite failures, etc.) + +1. *At most once*: Each record will be either processed once or not processed at all. +2. *At least once*: Each record will be processed one or more times. This is stronger than *at-most once* as it ensure that no data will be lost. But there may be duplicates. +3. *Exactly once*: Each record will be processed exactly once - no data will be lost and no data will be processed multiple times. This is obviously the strongest guarantee of the three. + +## Basic Semantics +{:.no_toc} +In any stream processing system, broadly speaking, there are three steps in processing the data. + +1. *Receiving the data*: The data is received from sources using Receivers or otherwise. + +1. *Transforming the data*: The data received data is transformed using DStream and RDD transformations. + +1. *Pushing out the data*: The final transformed data is pushed out to external systems like file systems, databases, dashboards, etc. + +If a streaming application has to achieve end-to-end exactly-once guarantees, then each step has to provide exactly-once guarantee. That is, each record must be received exactly once, transformed exactly once, and pushed to downstream systems exactly once. Let's understand the semantics of these steps in the context of Spark Streaming. + +1. *Receiving the data*: Different input sources provided different guarantees. This is discussed in detail in the next subsection. + +1. *Transforming the data*: All data that has been received will be processed _exactly once_, thanks to the guarantees that RDDs provide. Even if there are failures, as long as the received input data is accessible, the final transformed RDDs will always have the same contents. + +1. *Pushing out the data*: Output operations by default ensure _at-least once_ semantics because it depends on the type of output operation (idempotent, or not) and the semantics of the downstream system (supports transactions or not). But users can implement their own transaction mechanisms to achieve _exactly-once_ semantics. This is discussed in more details later in the section. + +## Semantics of Received Data +{:.no_toc} +Different input sources provide different guarantees, ranging from _at-least once_ to _exactly once_. Read for more details. + +### With Files {:.no_toc} If all of the input data is already present in a fault-tolerant files system like HDFS, Spark Streaming can always recover from any failure and process all the data. This gives *exactly-once* semantics, that all the data will be processed exactly once no matter what fails. -## Semantics with input sources based on receivers +### With Receiver-based Sources {:.no_toc} For input sources based on receivers, the fault-tolerance semantics depend on both the failure scenario and the type of receiver. @@ -1893,10 +2186,9 @@ receivers, data received but not replicated can get lost. If the driver node fai then besides these losses, all the past data that was received and replicated in memory will be lost. This will affect the results of the stateful transformations. -To avoid this loss of past received data, Spark 1.2 introduces an experimental feature of _write +To avoid this loss of past received data, Spark 1.2 introduced _write ahead logs_ which saves the received data to fault-tolerant storage. With the [write ahead logs -enabled](#deploying-applications) and reliable receivers, there is zero data loss and -exactly-once semantics. +enabled](#deploying-applications) and reliable receivers, there is zero data loss. In terms of semantics, it provides at-least once guarantee. The following table summarizes the semantics under failures: @@ -1908,23 +2200,30 @@ The following table summarizes the semantics under failures: - Spark 1.1 or earlier, or
- Spark 1.2 without write ahead log + Spark 1.1 or earlier, OR
+ Spark 1.2 or later without write ahead logs Buffered data lost with unreliable receivers
- Zero data loss with reliable receivers and files
+ Zero data loss with reliable receivers
+ At-least once semantics Buffered data lost with unreliable receivers
Past data lost with all receivers
- Zero data loss with files - + Undefined semantics + - Spark 1.2 with write ahead log - Zero data loss with reliable receivers and files - Zero data loss with reliable receivers and files + Spark 1.2 or later with write ahead logs + + Zero data loss with reliable receivers
+ At-least once semantics + + + Zero data loss with reliable receivers and files
+ At-least once semantics + @@ -1933,17 +2232,24 @@ The following table summarizes the semantics under failures: +### With Kafka Direct API +{:.no_toc} +In Spark 1.3, we have introduced a new Kafka Direct API, which can ensure that all the Kafka data is received by Spark Streaming exactly once. Along with this, if you implement exactly-once output operation, you can achieve end-to-end exactly-once guarantees. This approach (experimental as of Spark 1.3) is further discussed in the [Kafka Integration Guide](streaming-kafka-integration.html). + ## Semantics of output operations {:.no_toc} -Since all data is modeled as RDDs with their lineage of deterministic operations, any recomputation - always leads to the same result. As a result, all DStream transformations are guaranteed to have - _exactly-once_ semantics. That is, the final transformed result will be same even if there were - was a worker node failure. However, output operations (like `foreachRDD`) have _at-least once_ - semantics, that is, the transformed data may get written to an external entity more than once in - the event of a worker failure. While this is acceptable for saving to HDFS using the - `saveAs***Files` operations (as the file will simply get over-written by the same data), - additional transactions-like mechanisms may be necessary to achieve exactly-once semantics - for output operations. +Output operations (like `foreachRDD`) have _at-least once_ semantics, that is, +the transformed data may get written to an external entity more than once in +the event of a worker failure. While this is acceptable for saving to file systems using the +`saveAs***Files` operations (as the file will simply get overwritten with the same data), +additional effort may be necessary to achieve exactly-once semantics. There are two approaches. + +- *Idempotent updates*: Multiple attempts always write the same data. For example, `saveAs***Files` always writes the same data to the generated files. + +- *Transactional updates*: All updates are made transactionally so that updates are made exactly once atomically. One way to do this would be the following. + + - Use the batch time (available in `foreachRDD`) and the partition index of the transformed RDD to create an identifier. This identifier uniquely identifies a blob data in the streaming application. + - Update external system with this blob transactionally (that is, exactly once, atomically) using the identifier. That is, if the identifier is not already committed, commit the partition data and the identifier atomically. Else if this was already committed, skip the update. *************************************************************************************************** @@ -2001,7 +2307,11 @@ package and renamed for better clarity. *************************************************************************************************** # Where to Go from Here - +* Additional guides + - [Kafka Integration Guide](streaming-kafka-integration.html) + - [Flume Integration Guide](streaming-flume-integration.html) + - [Kinesis Integration Guide](streaming-kinesis-integration.html) + - [Custom Receiver Guide](streaming-custom-receivers.html) * API documentation - Scala docs * [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) and @@ -2023,8 +2333,8 @@ package and renamed for better clarity. [ZeroMQUtils](api/java/index.html?org/apache/spark/streaming/zeromq/ZeroMQUtils.html), and [MQTTUtils](api/java/index.html?org/apache/spark/streaming/mqtt/MQTTUtils.html) - Python docs - * [StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext) - * [DStream](api/python/pyspark.streaming.html#pyspark.streaming.DStream) + * [StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext) and [DStream](api/python/pyspark.streaming.html#pyspark.streaming.DStream) + * [KafkaUtils](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils) * More examples in [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming) and [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/streaming) diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh index 0857657152ec7..4f3e8da809f7f 100644 --- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh +++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh @@ -25,7 +25,6 @@ export MAPRED_LOCAL_DIRS="{{mapred_local_dirs}}" export SPARK_LOCAL_DIRS="{{spark_local_dirs}}" export MODULES="{{modules}}" export SPARK_VERSION="{{spark_version}}" -export SHARK_VERSION="{{shark_version}}" export TACHYON_VERSION="{{tachyon_version}}" export HADOOP_MAJOR_VERSION="{{hadoop_major_version}}" export SWAP_MB="{{swap}}" diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index f848874b0c775..c467cd08ed742 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -1159,8 +1159,8 @@ def real_main(): if EC2_INSTANCE_TYPES[opts.instance_type] != \ EC2_INSTANCE_TYPES[opts.master_instance_type]: print >> stderr, \ - "Error: spark-ec2 currently does not support having a master and slaves with " + \ - "different AMI virtualization types." + "Error: spark-ec2 currently does not support having a master and slaves " + \ + "with different AMI virtualization types." print >> stderr, "master instance virtualization type: {t}".format( t=EC2_INSTANCE_TYPES[opts.master_instance_type]) print >> stderr, "slave instance virtualization type: {t}".format( diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecord.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecord.java new file mode 100644 index 0000000000000..e63697a79f23a --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecord.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.streaming; + +/** Java Bean class to be used with the example JavaSqlNetworkWordCount. */ +public class JavaRecord implements java.io.Serializable { + private String word; + + public String getWord() { + return word; + } + + public void setWord(String word) { + this.word = word; + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java new file mode 100644 index 0000000000000..46562ddbbcb57 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaSqlNetworkWordCount.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.streaming; + +import java.util.regex.Pattern; + +import com.google.common.collect.Lists; + +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.api.java.StorageLevels; +import org.apache.spark.streaming.Durations; +import org.apache.spark.streaming.Time; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; + +/** + * Use DataFrames and SQL to count words in UTF8 encoded, '\n' delimited text received from the + * network every second. + * + * Usage: JavaSqlNetworkWordCount + * and describe the TCP server that Spark Streaming would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ bin/run-example org.apache.spark.examples.streaming.JavaSqlNetworkWordCount localhost 9999` + */ + +public final class JavaSqlNetworkWordCount { + private static final Pattern SPACE = Pattern.compile(" "); + + public static void main(String[] args) { + if (args.length < 2) { + System.err.println("Usage: JavaNetworkWordCount "); + System.exit(1); + } + + StreamingExamples.setStreamingLogLevels(); + + // Create the context with a 1 second batch size + SparkConf sparkConf = new SparkConf().setAppName("JavaSqlNetworkWordCount"); + JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, Durations.seconds(1)); + + // Create a JavaReceiverInputDStream on target ip:port and count the + // words in input stream of \n delimited text (eg. generated by 'nc') + // Note that no duplication in storage level only for running locally. + // Replication necessary in distributed scenario for fault tolerance. + JavaReceiverInputDStream lines = ssc.socketTextStream( + args[0], Integer.parseInt(args[1]), StorageLevels.MEMORY_AND_DISK_SER); + JavaDStream words = lines.flatMap(new FlatMapFunction() { + @Override + public Iterable call(String x) { + return Lists.newArrayList(SPACE.split(x)); + } + }); + + // Convert RDDs of the words DStream to DataFrame and run SQL query + words.foreachRDD(new Function2, Time, Void>() { + @Override + public Void call(JavaRDD rdd, Time time) { + SQLContext sqlContext = JavaSQLContextSingleton.getInstance(rdd.context()); + + // Convert JavaRDD[String] to JavaRDD[bean class] to DataFrame + JavaRDD rowRDD = rdd.map(new Function() { + public JavaRecord call(String word) { + JavaRecord record = new JavaRecord(); + record.setWord(word); + return record; + } + }); + DataFrame wordsDataFrame = sqlContext.createDataFrame(rowRDD, JavaRecord.class); + + // Register as table + wordsDataFrame.registerTempTable("words"); + + // Do word count on table using SQL and print it + DataFrame wordCountsDataFrame = + sqlContext.sql("select word, count(*) as total from words group by word"); + System.out.println("========= " + time + "========="); + wordCountsDataFrame.show(); + return null; + } + }); + + ssc.start(); + ssc.awaitTermination(); + } +} + +/** Lazily instantiated singleton instance of SQLContext */ +class JavaSQLContextSingleton { + static private transient SQLContext instance = null; + static public SQLContext getInstance(SparkContext sparkContext) { + if (instance == null) { + instance = new SQLContext(sparkContext); + } + return instance; + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java index d46c7107c7a21..dbf2ef02d7b76 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java @@ -82,6 +82,7 @@ public Optional call(List values, Optional state) { ssc.checkpoint("."); // Initial RDD input to updateStateByKey + @SuppressWarnings("unchecked") List> tuples = Arrays.asList(new Tuple2("hello", 1), new Tuple2("world", 1)); JavaPairRDD initialRDD = ssc.sc().parallelizePairs(tuples); diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py new file mode 100644 index 0000000000000..f89bc562d856b --- /dev/null +++ b/examples/src/main/python/streaming/sql_network_wordcount.py @@ -0,0 +1,82 @@ +# +# 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. +# + +""" + Use DataFrames and SQL to count words in UTF8 encoded, '\n' delimited text received from the + network every second. + + Usage: sql_network_wordcount.py + and describe the TCP server that Spark Streaming would connect to receive data. + + To run this on your local machine, you need to first run a Netcat server + `$ nc -lk 9999` + and then run the example + `$ bin/spark-submit examples/src/main/python/streaming/sql_network_wordcount.py localhost 9999` +""" + +import os +import sys + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext +from pyspark.sql import SQLContext, Row + + +def getSqlContextInstance(sparkContext): + if ('sqlContextSingletonInstance' not in globals()): + globals()['sqlContextSingletonInstance'] = SQLContext(sparkContext) + return globals()['sqlContextSingletonInstance'] + + +if __name__ == "__main__": + if len(sys.argv) != 3: + print >> sys.stderr, "Usage: sql_network_wordcount.py " + exit(-1) + host, port = sys.argv[1:] + sc = SparkContext(appName="PythonSqlNetworkWordCount") + ssc = StreamingContext(sc, 1) + + # Create a socket stream on target ip:port and count the + # words in input stream of \n delimited text (eg. generated by 'nc') + lines = ssc.socketTextStream(host, int(port)) + words = lines.flatMap(lambda line: line.split(" ")) + + # Convert RDDs of the words DStream to DataFrame and run SQL query + def process(time, rdd): + print "========= %s =========" % str(time) + + try: + # Get the singleton instance of SQLContext + sqlContext = getSqlContextInstance(rdd.context) + + # Convert RDD[String] to RDD[Row] to DataFrame + rowRdd = rdd.map(lambda w: Row(word=w)) + wordsDataFrame = sqlContext.createDataFrame(rowRdd) + + # Register as table + wordsDataFrame.registerTempTable("words") + + # Do word count on table using SQL and print it + wordCountsDataFrame = \ + sqlContext.sql("select word, count(*) as total from words group by word") + wordCountsDataFrame.show() + except: + pass + + words.foreachRDD(process) + ssc.start() + ssc.awaitTermination() diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala index 822673347bdce..f4684b42b5d41 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala @@ -18,7 +18,7 @@ package org.apache.spark.examples import org.apache.hadoop.hbase.client.HBaseAdmin -import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor} +import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor, TableName} import org.apache.hadoop.hbase.mapreduce.TableInputFormat import org.apache.spark._ @@ -36,7 +36,7 @@ object HBaseTest { // Initialize hBase table if necessary val admin = new HBaseAdmin(conf) if (!admin.isTableAvailable(args(0))) { - val tableDesc = new HTableDescriptor(args(0)) + val tableDesc = new HTableDescriptor(TableName.valueOf(args(0))) admin.createTable(tableDesc) } diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala new file mode 100644 index 0000000000000..5a6b9216a3fbc --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.streaming + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Time, Seconds, StreamingContext} +import org.apache.spark.util.IntParam +import org.apache.spark.sql.SQLContext +import org.apache.spark.storage.StorageLevel + +/** + * Use DataFrames and SQL to count words in UTF8 encoded, '\n' delimited text received from the + * network every second. + * + * Usage: SqlNetworkWordCount + * and describe the TCP server that Spark Streaming would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ bin/run-example org.apache.spark.examples.streaming.SqlNetworkWordCount localhost 9999` + */ + +object SqlNetworkWordCount { + def main(args: Array[String]) { + if (args.length < 2) { + System.err.println("Usage: NetworkWordCount ") + System.exit(1) + } + + StreamingExamples.setStreamingLogLevels() + + // Create the context with a 2 second batch size + val sparkConf = new SparkConf().setAppName("SqlNetworkWordCount") + val ssc = new StreamingContext(sparkConf, Seconds(2)) + + // Create a socket stream on target ip:port and count the + // words in input stream of \n delimited text (eg. generated by 'nc') + // Note that no duplication in storage level only for running locally. + // Replication necessary in distributed scenario for fault tolerance. + val lines = ssc.socketTextStream(args(0), args(1).toInt, StorageLevel.MEMORY_AND_DISK_SER) + val words = lines.flatMap(_.split(" ")) + + // Convert RDDs of the words DStream to DataFrame and run SQL query + words.foreachRDD((rdd: RDD[String], time: Time) => { + // Get the singleton instance of SQLContext + val sqlContext = SQLContextSingleton.getInstance(rdd.sparkContext) + import sqlContext.implicits._ + + // Convert RDD[String] to RDD[case class] to DataFrame + val wordsDataFrame = rdd.map(w => Record(w)).toDF() + + // Register as table + wordsDataFrame.registerTempTable("words") + + // Do word count on table using SQL and print it + val wordCountsDataFrame = + sqlContext.sql("select word, count(*) as total from words group by word") + println(s"========= $time =========") + wordCountsDataFrame.show() + }) + + ssc.start() + ssc.awaitTermination() + } +} + + +/** Case class for converting RDD to DataFrame */ +case class Record(word: String) + + +/** Lazily instantiated singleton instance of SQLContext */ +object SQLContextSingleton { + + @transient private var instance: SQLContext = _ + + def getInstance(sparkContext: SparkContext): SQLContext = { + if (instance == null) { + instance = new SQLContext(sparkContext) + } + instance + } +} diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index d56cc01be9514..6d465bcb6bfc0 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -86,7 +86,7 @@ class KafkaRDD[ val part = thePart.asInstanceOf[KafkaRDDPartition] assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part)) if (part.fromOffset == part.untilOffset) { - log.warn("Beginning offset ${part.fromOffset} is the same as ending offset " + + log.warn(s"Beginning offset ${part.fromOffset} is the same as ending offset " + s"skipping ${part.topic} ${part.partition}") Iterator.empty } else { diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java index 1334cc8fd1b57..d6ca6d58b5665 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -20,32 +20,27 @@ import java.io.Serializable; import java.util.HashMap; import java.util.HashSet; -import java.util.Random; import java.util.Arrays; -import org.apache.spark.SparkConf; - import scala.Tuple2; -import junit.framework.Assert; - import kafka.common.TopicAndPartition; import kafka.message.MessageAndMetadata; import kafka.serializer.StringDecoder; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.Function; -import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.Durations; +import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.junit.Test; -import org.junit.After; -import org.junit.Before; - public class JavaDirectKafkaStreamSuite implements Serializable { private transient JavaStreamingContext ssc = null; - private transient Random random = new Random(); private transient KafkaStreamSuiteBase suiteBase = null; @Before @@ -93,7 +88,7 @@ public void testKafkaStream() throws InterruptedException { ).map( new Function, String>() { @Override - public String call(scala.Tuple2 kv) throws Exception { + public String call(Tuple2 kv) throws Exception { return kv._2(); } } @@ -121,7 +116,7 @@ public String call(MessageAndMetadata msgAndMd) throws Exception unifiedStream.foreachRDD( new Function, Void>() { @Override - public Void call(org.apache.spark.api.java.JavaRDD rdd) throws Exception { + public Void call(JavaRDD rdd) throws Exception { result.addAll(rdd.collect()); return null; } diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java index 9d2e1705c6c73..4477b81827c70 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java @@ -19,27 +19,22 @@ import java.io.Serializable; import java.util.HashMap; -import java.util.HashSet; -import java.util.Arrays; - -import org.apache.spark.SparkConf; import scala.Tuple2; -import junit.framework.Assert; - import kafka.common.TopicAndPartition; import kafka.message.MessageAndMetadata; import kafka.serializer.StringDecoder; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; -import org.junit.Test; -import org.junit.After; -import org.junit.Before; - public class JavaKafkaRDDSuite implements Serializable { private transient JavaSparkContext sc = null; private transient KafkaStreamSuiteBase suiteBase = null; @@ -78,8 +73,8 @@ public void testKafkaRDD() throws InterruptedException { OffsetRange.create(topic2, 0, 0, 1) }; - HashMap emptyLeaders = new HashMap(); - HashMap leaders = new HashMap(); + HashMap emptyLeaders = new HashMap(); + HashMap leaders = new HashMap(); String[] hostAndPort = suiteBase.brokerAddress().split(":"); Broker broker = Broker.create(hostAndPort[0], Integer.parseInt(hostAndPort[1])); leaders.put(new TopicAndPartition(topic1, 0), broker); @@ -96,7 +91,7 @@ public void testKafkaRDD() throws InterruptedException { ).map( new Function, String>() { @Override - public String call(scala.Tuple2 kv) throws Exception { + public String call(Tuple2 kv) throws Exception { return kv._2(); } } diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 208cc51b29876..bad0a93eb2e84 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -22,27 +22,25 @@ import java.util.List; import java.util.Random; -import org.apache.spark.SparkConf; -import org.apache.spark.streaming.Duration; import scala.Predef; import scala.Tuple2; import scala.collection.JavaConverters; -import junit.framework.Assert; - import kafka.serializer.StringDecoder; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.function.Function; import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.junit.Test; -import org.junit.After; -import org.junit.Before; - public class JavaKafkaStreamSuite implements Serializable { private transient JavaStreamingContext ssc = null; private transient Random random = new Random(); diff --git a/graphx/pom.xml b/graphx/pom.xml index 57e338c03ecf9..c0d534e185d7f 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -45,9 +45,14 @@ guava - org.jblas - jblas - ${jblas.version} + com.github.fommil.netlib + core + ${netlib.java.version} + + + net.sourceforge.f2j + arpack_combined_all + 0.1 org.scalacheck diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala index f70715fca6eea..d8be02e2023d5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeContext.scala @@ -49,3 +49,20 @@ abstract class EdgeContext[VD, ED, A] { et } } + +object EdgeContext { + + /** + * Extractor mainly used for Graph#aggregateMessages*. + * Example: + * {{{ + * val messages = graph.aggregateMessages( + * case ctx @ EdgeContext(_, _, _, _, attr) => + * ctx.sendToDst(attr) + * , _ + _) + * }}} + */ + def unapply[VD, ED, A](edge: EdgeContext[VD, ED, A]) = + Some(edge.srcId, edge.dstId, edge.srcAttr, edge.dstAttr, edge.attr) +} + 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 09ae3f9f6c09b..ad4bfe077293a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -122,8 +122,20 @@ abstract class VertexRDD[VD]( def mapValues[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexRDD[VD2] /** - * Hides vertices that are the same between `this` and `other`; for vertices that are different, - * keeps the values from `other`. + * For each vertex present in both `this` and `other`, `diff` returns only those vertices with + * differing values; for values that are different, keeps the values from `other`. This is + * only guaranteed to work if the VertexRDDs share a common ancestor. + * + * @param other the other RDD[(VertexId, VD)] with which to diff against. + */ + def diff(other: RDD[(VertexId, VD)]): VertexRDD[VD] + + /** + * For each vertex present in both `this` and `other`, `diff` returns only those vertices with + * differing values; for values that are different, keeps the values from `other`. This is + * only guaranteed to work if the VertexRDDs share a common ancestor. + * + * @param other the other VertexRDD with which to diff against. */ def diff(other: VertexRDD[VD]): VertexRDD[VD] diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala index 904be213147dc..125692ddaad83 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala @@ -103,6 +103,10 @@ class VertexRDDImpl[VD] private[graphx] ( override def mapValues[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexRDD[VD2] = this.mapVertexPartitions(_.map(f)) + override def diff(other: RDD[(VertexId, VD)]): VertexRDD[VD] = { + diff(this.aggregateUsingIndex(other, (a: VD, b: VD) => a)) + } + override def diff(other: VertexRDD[VD]): VertexRDD[VD] = { val otherPartition = other match { case other: VertexRDD[_] if this.partitioner == other.partitioner => diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala index 3e4157a63fd1c..1a7178b82e3af 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala @@ -18,7 +18,9 @@ package org.apache.spark.graphx.lib import scala.util.Random -import org.jblas.DoubleMatrix + +import com.github.fommil.netlib.BLAS.{getInstance => blas} + import org.apache.spark.rdd._ import org.apache.spark.graphx._ @@ -53,7 +55,7 @@ object SVDPlusPlus { * a Multifaceted Collaborative Filtering Model", * available at [[http://public.research.att.com/~volinsky/netflix/kdd08koren.pdf]]. * - * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^(-0.5)*sum(y)), + * The prediction rule is rui = u + bu + bi + qi*(pu + |N(u)|^^-0.5^^*sum(y)), * see the details on page 6. * * @param edges edges for constructing the graph @@ -66,13 +68,10 @@ object SVDPlusPlus { : (Graph[(Array[Double], Array[Double], Double, Double), Double], Double) = { // Generate default vertex attribute - def defaultF(rank: Int): (DoubleMatrix, DoubleMatrix, Double, Double) = { - val v1 = new DoubleMatrix(rank) - val v2 = new DoubleMatrix(rank) - for (i <- 0 until rank) { - v1.put(i, Random.nextDouble()) - v2.put(i, Random.nextDouble()) - } + def defaultF(rank: Int): (Array[Double], Array[Double], Double, Double) = { + // TODO: use a fixed random seed + val v1 = Array.fill(rank)(Random.nextDouble()) + val v2 = Array.fill(rank)(Random.nextDouble()) (v1, v2, 0.0, 0.0) } @@ -92,7 +91,7 @@ object SVDPlusPlus { (g1, g2) => (g1._1 + g2._1, g1._2 + g2._2)) val gJoinT0 = g.outerJoinVertices(t0) { - (vid: VertexId, vd: (DoubleMatrix, DoubleMatrix, Double, Double), + (vid: VertexId, vd: (Array[Double], Array[Double], Double, Double), msg: Option[(Long, Double)]) => (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1)) }.cache() @@ -102,24 +101,28 @@ object SVDPlusPlus { def sendMsgTrainF(conf: Conf, u: Double) (ctx: EdgeContext[ - (DoubleMatrix, DoubleMatrix, Double, Double), + (Array[Double], Array[Double], Double, Double), Double, - (DoubleMatrix, DoubleMatrix, Double)]) { + (Array[Double], Array[Double], Double)]) { val (usr, itm) = (ctx.srcAttr, ctx.dstAttr) val (p, q) = (usr._1, itm._1) - var pred = u + usr._3 + itm._3 + q.dot(usr._2) + val rank = p.length + var pred = u + usr._3 + itm._3 + blas.ddot(rank, q, 1, usr._2, 1) pred = math.max(pred, conf.minVal) pred = math.min(pred, conf.maxVal) val err = ctx.attr - pred - val updateP = q.mul(err) - .subColumnVector(p.mul(conf.gamma7)) - .mul(conf.gamma2) - val updateQ = usr._2.mul(err) - .subColumnVector(q.mul(conf.gamma7)) - .mul(conf.gamma2) - val updateY = q.mul(err * usr._4) - .subColumnVector(itm._2.mul(conf.gamma7)) - .mul(conf.gamma2) + // updateP = (err * q - conf.gamma7 * p) * conf.gamma2 + val updateP = q.clone() + blas.dscal(rank, err * conf.gamma2, updateP, 1) + blas.daxpy(rank, -conf.gamma7 * conf.gamma2, p, 1, updateP, 1) + // updateQ = (err * usr._2 - conf.gamma7 * q) * conf.gamma2 + val updateQ = usr._2.clone() + blas.dscal(rank, err * conf.gamma2, updateQ, 1) + blas.daxpy(rank, -conf.gamma7 * conf.gamma2, q, 1, updateQ, 1) + // updateY = (err * usr._4 * q - conf.gamma7 * itm._2) * conf.gamma2 + val updateY = q.clone() + blas.dscal(rank, err * usr._4 * conf.gamma2, updateY, 1) + blas.daxpy(rank, -conf.gamma7 * conf.gamma2, itm._2, 1, updateY, 1) ctx.sendToSrc((updateP, updateY, (err - conf.gamma6 * usr._3) * conf.gamma1)) ctx.sendToDst((updateQ, updateY, (err - conf.gamma6 * itm._3) * conf.gamma1)) } @@ -127,14 +130,23 @@ object SVDPlusPlus { for (i <- 0 until conf.maxIters) { // Phase 1, calculate pu + |N(u)|^(-0.5)*sum(y) for user nodes g.cache() - val t1 = g.aggregateMessages[DoubleMatrix]( + val t1 = g.aggregateMessages[Array[Double]]( ctx => ctx.sendToSrc(ctx.dstAttr._2), - (g1, g2) => g1.addColumnVector(g2)) + (g1, g2) => { + val out = g1.clone() + blas.daxpy(out.length, 1.0, g2, 1, out, 1) + out + }) val gJoinT1 = g.outerJoinVertices(t1) { - (vid: VertexId, vd: (DoubleMatrix, DoubleMatrix, Double, Double), - msg: Option[DoubleMatrix]) => - if (msg.isDefined) (vd._1, vd._1 - .addColumnVector(msg.get.mul(vd._4)), vd._3, vd._4) else vd + (vid: VertexId, vd: (Array[Double], Array[Double], Double, Double), + msg: Option[Array[Double]]) => + if (msg.isDefined) { + val out = vd._1.clone() + blas.daxpy(out.length, vd._4, msg.get, 1, out, 1) + (vd._1, out, vd._3, vd._4) + } else { + vd + } }.cache() materialize(gJoinT1) g.unpersist() @@ -144,14 +156,24 @@ object SVDPlusPlus { g.cache() val t2 = g.aggregateMessages( sendMsgTrainF(conf, u), - (g1: (DoubleMatrix, DoubleMatrix, Double), g2: (DoubleMatrix, DoubleMatrix, Double)) => - (g1._1.addColumnVector(g2._1), g1._2.addColumnVector(g2._2), g1._3 + g2._3)) + (g1: (Array[Double], Array[Double], Double), g2: (Array[Double], Array[Double], Double)) => + { + val out1 = g1._1.clone() + blas.daxpy(out1.length, 1.0, g2._1, 1, out1, 1) + val out2 = g2._2.clone() + blas.daxpy(out2.length, 1.0, g2._2, 1, out2, 1) + (out1, out2, g1._3 + g2._3) + }) val gJoinT2 = g.outerJoinVertices(t2) { (vid: VertexId, - vd: (DoubleMatrix, DoubleMatrix, Double, Double), - msg: Option[(DoubleMatrix, DoubleMatrix, Double)]) => - (vd._1.addColumnVector(msg.get._1), vd._2.addColumnVector(msg.get._2), - vd._3 + msg.get._3, vd._4) + vd: (Array[Double], Array[Double], Double, Double), + msg: Option[(Array[Double], Array[Double], Double)]) => { + val out1 = vd._1.clone() + blas.daxpy(out1.length, 1.0, msg.get._1, 1, out1, 1) + val out2 = vd._2.clone() + blas.daxpy(out2.length, 1.0, msg.get._2, 1, out2, 1) + (out1, out2, vd._3 + msg.get._3, vd._4) + } }.cache() materialize(gJoinT2) g.unpersist() @@ -160,10 +182,10 @@ object SVDPlusPlus { // calculate error on training set def sendMsgTestF(conf: Conf, u: Double) - (ctx: EdgeContext[(DoubleMatrix, DoubleMatrix, Double, Double), Double, Double]) { + (ctx: EdgeContext[(Array[Double], Array[Double], Double, Double), Double, Double]) { val (usr, itm) = (ctx.srcAttr, ctx.dstAttr) val (p, q) = (usr._1, itm._1) - var pred = u + usr._3 + itm._3 + q.dot(usr._2) + var pred = u + usr._3 + itm._3 + blas.ddot(q.length, q, 1, usr._2, 1) pred = math.max(pred, conf.minVal) pred = math.min(pred, conf.maxVal) val err = (ctx.attr - pred) * (ctx.attr - pred) @@ -173,7 +195,7 @@ object SVDPlusPlus { g.cache() val t3 = g.aggregateMessages[Double](sendMsgTestF(conf, u), _ + _) val gJoinT3 = g.outerJoinVertices(t3) { - (vid: VertexId, vd: (DoubleMatrix, DoubleMatrix, Double, Double), msg: Option[Double]) => + (vid: VertexId, vd: (Array[Double], Array[Double], Double, Double), msg: Option[Double]) => if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd }.cache() materialize(gJoinT3) 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 131959cea3ef7..4f7a442ab503d 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.graphx import org.scalatest.FunSuite -import org.apache.spark.SparkContext +import org.apache.spark.{HashPartitioner, SparkContext} +import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel class VertexRDDSuite extends FunSuite with LocalSparkContext { @@ -58,6 +59,28 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } + test("diff with RDD[(VertexId, VD)]") { + withSpark { sc => + val n = 100 + val verts = vertices(sc, n).cache() + val flipEvens: RDD[(VertexId, Int)] = + sc.parallelize(0L to 100L) + .map(id => if (id % 2 == 0) (id, -id.toInt) else (id, id.toInt)).cache() + // diff should keep only the changed vertices + assert(verts.diff(flipEvens).map(_._2).collect().toSet === (2 to n by 2).map(-_).toSet) + } + } + + test("diff vertices with the non-equal number of partitions") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 24, 3).map(i => (i.toLong, 0))) + val vertexB = VertexRDD(sc.parallelize(8 until 16, 2).map(i => (i.toLong, 1))) + assert(vertexA.partitions.size != vertexB.partitions.size) + val vertexC = vertexA.diff(vertexB) + assert(vertexC.map(_._1).collect.toSet === (8 until 16).toSet) + } + } + test("leftJoin") { withSpark { sc => val n = 100 @@ -73,6 +96,19 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } + test("leftJoin vertices with the non-equal number of partitions") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 100, 2).map(i => (i.toLong, 1))) + val vertexB = VertexRDD( + vertexA.filter(v => v._1 % 2 == 0).partitionBy(new HashPartitioner(3))) + assert(vertexA.partitions.size != vertexB.partitions.size) + val vertexC = vertexA.leftJoin(vertexB) { (vid, old, newOpt) => + old - newOpt.getOrElse(0) + } + assert(vertexC.filter(v => v._2 != 0).map(_._1).collect.toSet == (1 to 99 by 2).toSet) + } + } + test("innerJoin") { withSpark { sc => val n = 100 @@ -87,6 +123,19 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { (0 to n by 2).map(x => (x.toLong, 0)).toSet) } } + test("innerJoin vertices with the non-equal number of partitions") { + withSpark { sc => + val vertexA = VertexRDD(sc.parallelize(0 until 100, 2).map(i => (i.toLong, 1))) + val vertexB = VertexRDD( + vertexA.filter(v => v._1 % 2 == 0).partitionBy(new HashPartitioner(3))) + assert(vertexA.partitions.size != vertexB.partitions.size) + val vertexC = vertexA.innerJoin(vertexB) { (vid, old, newVal) => + old - newVal + } + assert(vertexC.filter(v => v._2 == 0).map(_._1).collect.toSet == (0 to 98 by 2).toSet) + } + } + test("aggregateUsingIndex") { withSpark { sc => val n = 100 diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala index 9987a4b1a3c25..7bd6b7f3c4ab2 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala @@ -32,11 +32,11 @@ class SVDPlusPlusSuite extends FunSuite with LocalSparkContext { Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble) } val conf = new SVDPlusPlus.Conf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations - var (graph, u) = SVDPlusPlus.runSVDPlusPlus(edges, conf) + val (graph, _) = SVDPlusPlus.run(edges, conf) graph.cache() - val err = graph.vertices.collect().map{ case (vid, vd) => + val err = graph.vertices.map { case (vid, vd) => if (vid % 2 == 1) vd._4 else 0.0 - }.reduce(_ + _) / graph.triplets.collect().size + }.reduce(_ + _) / graph.numEdges assert(err <= svdppErr) } } diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index 6ffdff63d3c78..91dcf70f105db 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -253,12 +253,6 @@ private boolean isClientMode(Properties userProps) { private class OptionParser extends SparkSubmitOptionParser { - private final List driverJvmKeys = Arrays.asList( - SparkLauncher.DRIVER_EXTRA_CLASSPATH, - SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, - SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, - SparkLauncher.DRIVER_MEMORY); - @Override protected boolean handle(String opt, String value) { if (opt.equals(MASTER)) { @@ -278,9 +272,7 @@ protected boolean handle(String opt, String value) { } else if (opt.equals(CONF)) { String[] setConf = value.split("=", 2); checkArgument(setConf.length == 2, "Invalid argument to %s: %s", CONF, value); - if (driverJvmKeys.contains(setConf[0])) { - conf.put(setConf[0], setConf[1]); - } + conf.put(setConf[0], setConf[1]); } else if (opt.equals(CLASS)) { // The special classes require some special command line handling, since they allow // mixing spark-submit arguments with arguments that should be propagated to the shell diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index 815edc4e4971f..626116a9e7477 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -68,6 +68,8 @@ public void testCliParser() throws Exception { parser.DRIVER_JAVA_OPTIONS, "extraJavaOpt", parser.CONF, + "spark.randomOption=foo", + parser.CONF, SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH + "=/driverLibPath"); Map env = new HashMap(); List cmd = buildCommand(sparkSubmitArgs, env); @@ -77,6 +79,8 @@ public void testCliParser() throws Exception { assertTrue(findInStringList(findArgValue(cmd, "-cp"), File.pathSeparator, "/driverCp")); assertTrue("Driver -Xms should be configured.", cmd.contains("-Xms42g")); assertTrue("Driver -Xmx should be configured.", cmd.contains("-Xmx42g")); + assertTrue("Command should contain user-defined conf.", + Collections.indexOfSubList(cmd, Arrays.asList(parser.CONF, "spark.randomOption=foo")) > 0); } @Test diff --git a/make-distribution.sh b/make-distribution.sh index 82d33408cd5e5..9ed1abfe8c598 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -127,7 +127,9 @@ if [ ! $(command -v "$MVN") ] ; then fi VERSION=$("$MVN" help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1) -SCALA_VERSION=$("$MVN" help:evaluate -Dexpression=scala.binary.version 2>/dev/null | grep -v "INFO" | tail -n 1) +SCALA_VERSION=$("$MVN" help:evaluate -Dexpression=scala.binary.version $@ 2>/dev/null\ + | grep -v "INFO"\ + | tail -n 1) SPARK_HADOOP_VERSION=$("$MVN" help:evaluate -Dexpression=hadoop.version $@ 2>/dev/null\ | grep -v "INFO"\ | tail -n 1) diff --git a/mllib/pom.xml b/mllib/pom.xml index b5c949e155cfd..a76704a8c2c59 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -59,6 +59,7 @@ org.jblas jblas ${jblas.version} + test org.scalanlp @@ -116,7 +117,7 @@ com.github.fommil.netlib all - 1.1.2 + ${netlib.java.version} pom diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala new file mode 100644 index 0000000000000..970e6ad5514d1 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala @@ -0,0 +1,234 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.attribute + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.mllib.linalg.VectorUDT +import org.apache.spark.sql.types.{Metadata, MetadataBuilder, StructField} + +/** + * Attributes that describe a vector ML column. + * + * @param name name of the attribute group (the ML column name) + * @param numAttributes optional number of attributes. At most one of `numAttributes` and `attrs` + * can be defined. + * @param attrs optional array of attributes. Attribute will be copied with their corresponding + * indices in the array. + */ +class AttributeGroup private ( + val name: String, + val numAttributes: Option[Int], + attrs: Option[Array[Attribute]]) extends Serializable { + + require(name.nonEmpty, "Cannot have an empty string for name.") + require(!(numAttributes.isDefined && attrs.isDefined), + "Cannot have both numAttributes and attrs defined.") + + /** + * Creates an attribute group without attribute info. + * @param name name of the attribute group + */ + def this(name: String) = this(name, None, None) + + /** + * Creates an attribute group knowing only the number of attributes. + * @param name name of the attribute group + * @param numAttributes number of attributes + */ + def this(name: String, numAttributes: Int) = this(name, Some(numAttributes), None) + + /** + * Creates an attribute group with attributes. + * @param name name of the attribute group + * @param attrs array of attributes. Attributes will be copied with their corresponding indices in + * the array. + */ + def this(name: String, attrs: Array[Attribute]) = this(name, None, Some(attrs)) + + /** + * Optional array of attributes. At most one of `numAttributes` and `attributes` can be defined. + */ + val attributes: Option[Array[Attribute]] = attrs.map(_.view.zipWithIndex.map { case (attr, i) => + attr.withIndex(i) + }.toArray) + + private lazy val nameToIndex: Map[String, Int] = { + attributes.map(_.view.flatMap { attr => + attr.name.map(_ -> attr.index.get) + }.toMap).getOrElse(Map.empty) + } + + /** Size of the attribute group. Returns -1 if the size is unknown. */ + def size: Int = { + if (numAttributes.isDefined) { + numAttributes.get + } else if (attributes.isDefined) { + attributes.get.length + } else { + -1 + } + } + + /** Test whether this attribute group contains a specific attribute. */ + def hasAttr(attrName: String): Boolean = nameToIndex.contains(attrName) + + /** Index of an attribute specified by name. */ + def indexOf(attrName: String): Int = nameToIndex(attrName) + + /** Gets an attribute by its name. */ + def apply(attrName: String): Attribute = { + attributes.get(indexOf(attrName)) + } + + /** Gets an attribute by its name. */ + def getAttr(attrName: String): Attribute = this(attrName) + + /** Gets an attribute by its index. */ + def apply(attrIndex: Int): Attribute = attributes.get(attrIndex) + + /** Gets an attribute by its index. */ + def getAttr(attrIndex: Int): Attribute = this(attrIndex) + + /** Converts to metadata without name. */ + private[attribute] def toMetadata: Metadata = { + import AttributeKeys._ + val bldr = new MetadataBuilder() + if (attributes.isDefined) { + val numericMetadata = ArrayBuffer.empty[Metadata] + val nominalMetadata = ArrayBuffer.empty[Metadata] + val binaryMetadata = ArrayBuffer.empty[Metadata] + attributes.get.foreach { + case numeric: NumericAttribute => + // Skip default numeric attributes. + if (numeric.withoutIndex != NumericAttribute.defaultAttr) { + numericMetadata += numeric.toMetadata(withType = false) + } + case nominal: NominalAttribute => + nominalMetadata += nominal.toMetadata(withType = false) + case binary: BinaryAttribute => + binaryMetadata += binary.toMetadata(withType = false) + } + val attrBldr = new MetadataBuilder + if (numericMetadata.nonEmpty) { + attrBldr.putMetadataArray(AttributeType.Numeric.name, numericMetadata.toArray) + } + if (nominalMetadata.nonEmpty) { + attrBldr.putMetadataArray(AttributeType.Nominal.name, nominalMetadata.toArray) + } + if (binaryMetadata.nonEmpty) { + attrBldr.putMetadataArray(AttributeType.Binary.name, binaryMetadata.toArray) + } + bldr.putMetadata(ATTRIBUTES, attrBldr.build()) + bldr.putLong(NUM_ATTRIBUTES, attributes.get.length) + } else if (numAttributes.isDefined) { + bldr.putLong(NUM_ATTRIBUTES, numAttributes.get) + } + bldr.build() + } + + /** Converts to a StructField with some existing metadata. */ + def toStructField(existingMetadata: Metadata): StructField = { + val newMetadata = new MetadataBuilder() + .withMetadata(existingMetadata) + .putMetadata(AttributeKeys.ML_ATTR, toMetadata) + .build() + StructField(name, new VectorUDT, nullable = false, newMetadata) + } + + /** Converts to a StructField. */ + def toStructField(): StructField = toStructField(Metadata.empty) + + override def equals(other: Any): Boolean = { + other match { + case o: AttributeGroup => + (name == o.name) && + (numAttributes == o.numAttributes) && + (attributes.map(_.toSeq) == o.attributes.map(_.toSeq)) + case _ => + false + } + } + + override def hashCode: Int = { + var sum = 17 + sum = 37 * sum + name.hashCode + sum = 37 * sum + numAttributes.hashCode + sum = 37 * sum + attributes.map(_.toSeq).hashCode + sum + } +} + +/** Factory methods to create attribute groups. */ +object AttributeGroup { + + import AttributeKeys._ + + /** Creates an attribute group from a [[Metadata]] instance with name. */ + private[attribute] def fromMetadata(metadata: Metadata, name: String): AttributeGroup = { + import org.apache.spark.ml.attribute.AttributeType._ + if (metadata.contains(ATTRIBUTES)) { + val numAttrs = metadata.getLong(NUM_ATTRIBUTES).toInt + val attributes = new Array[Attribute](numAttrs) + val attrMetadata = metadata.getMetadata(ATTRIBUTES) + if (attrMetadata.contains(Numeric.name)) { + attrMetadata.getMetadataArray(Numeric.name) + .map(NumericAttribute.fromMetadata) + .foreach { attr => + attributes(attr.index.get) = attr + } + } + if (attrMetadata.contains(Nominal.name)) { + attrMetadata.getMetadataArray(Nominal.name) + .map(NominalAttribute.fromMetadata) + .foreach { attr => + attributes(attr.index.get) = attr + } + } + if (attrMetadata.contains(Binary.name)) { + attrMetadata.getMetadataArray(Binary.name) + .map(BinaryAttribute.fromMetadata) + .foreach { attr => + attributes(attr.index.get) = attr + } + } + var i = 0 + while (i < numAttrs) { + if (attributes(i) == null) { + attributes(i) = NumericAttribute.defaultAttr + } + i += 1 + } + new AttributeGroup(name, attributes) + } else if (metadata.contains(NUM_ATTRIBUTES)) { + new AttributeGroup(name, metadata.getLong(NUM_ATTRIBUTES).toInt) + } else { + new AttributeGroup(name) + } + } + + /** Creates an attribute group from a [[StructField]] instance. */ + def fromStructField(field: StructField): AttributeGroup = { + require(field.dataType == new VectorUDT) + if (field.metadata.contains(ML_ATTR)) { + fromMetadata(field.metadata.getMetadata(ML_ATTR), field.name) + } else { + new AttributeGroup(field.name) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeKeys.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeKeys.scala new file mode 100644 index 0000000000000..f714f7becc7e6 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeKeys.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.attribute + +/** + * Keys used to store attributes. + */ +private[attribute] object AttributeKeys { + val ML_ATTR: String = "ml_attr" + val TYPE: String = "type" + val NAME: String = "name" + val INDEX: String = "idx" + val MIN: String = "min" + val MAX: String = "max" + val STD: String = "std" + val SPARSITY: String = "sparsity" + val ORDINAL: String = "ord" + val VALUES: String = "vals" + val NUM_VALUES: String = "num_vals" + val ATTRIBUTES: String = "attrs" + val NUM_ATTRIBUTES: String = "num_attrs" +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeType.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeType.scala new file mode 100644 index 0000000000000..65e7e43d5a5b0 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeType.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.attribute + +/** + * An enum-like type for attribute types: [[AttributeType$#Numeric]], [[AttributeType$#Nominal]], + * and [[AttributeType$#Binary]]. + */ +sealed abstract class AttributeType(val name: String) + +object AttributeType { + + /** Numeric type. */ + val Numeric: AttributeType = { + case object Numeric extends AttributeType("numeric") + Numeric + } + + /** Nominal type. */ + val Nominal: AttributeType = { + case object Nominal extends AttributeType("nominal") + Nominal + } + + /** Binary type. */ + val Binary: AttributeType = { + case object Binary extends AttributeType("binary") + Binary + } + + /** + * Gets the [[AttributeType]] object from its name. + * @param name attribute type name: "numeric", "nominal", or "binary" + */ + def fromName(name: String): AttributeType = { + if (name == Numeric.name) { + Numeric + } else if (name == Nominal.name) { + Nominal + } else if (name == Binary.name) { + Binary + } else { + throw new IllegalArgumentException(s"Cannot recognize type $name.") + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala new file mode 100644 index 0000000000000..00b7566aab434 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala @@ -0,0 +1,512 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.attribute + +import scala.annotation.varargs + +import org.apache.spark.sql.types.{DoubleType, Metadata, MetadataBuilder, StructField} + +/** + * Abstract class for ML attributes. + */ +sealed abstract class Attribute extends Serializable { + + name.foreach { n => + require(n.nonEmpty, "Cannot have an empty string for name.") + } + index.foreach { i => + require(i >= 0, s"Index cannot be negative but got $i") + } + + /** Attribute type. */ + def attrType: AttributeType + + /** Name of the attribute. None if it is not set. */ + def name: Option[String] + + /** Copy with a new name. */ + def withName(name: String): Attribute + + /** Copy without the name. */ + def withoutName: Attribute + + /** Index of the attribute. None if it is not set. */ + def index: Option[Int] + + /** Copy with a new index. */ + def withIndex(index: Int): Attribute + + /** Copy without the index. */ + def withoutIndex: Attribute + + /** + * Tests whether this attribute is numeric, true for [[NumericAttribute]] and [[BinaryAttribute]]. + */ + def isNumeric: Boolean + + /** + * Tests whether this attribute is nominal, true for [[NominalAttribute]] and [[BinaryAttribute]]. + */ + def isNominal: Boolean + + /** + * Converts this attribute to [[Metadata]]. + * @param withType whether to include the type info + */ + private[attribute] def toMetadata(withType: Boolean): Metadata + + /** + * Converts this attribute to [[Metadata]]. For numeric attributes, the type info is excluded to + * save space, because numeric type is the default attribute type. For nominal and binary + * attributes, the type info is included. + */ + private[attribute] def toMetadata(): Metadata = { + if (attrType == AttributeType.Numeric) { + toMetadata(withType = false) + } else { + toMetadata(withType = true) + } + } + + /** + * Converts to a [[StructField]] with some existing metadata. + * @param existingMetadata existing metadata to carry over + */ + def toStructField(existingMetadata: Metadata): StructField = { + val newMetadata = new MetadataBuilder() + .withMetadata(existingMetadata) + .putMetadata(AttributeKeys.ML_ATTR, withoutName.withoutIndex.toMetadata()) + .build() + StructField(name.get, DoubleType, nullable = false, newMetadata) + } + + /** Converts to a [[StructField]]. */ + def toStructField(): StructField = toStructField(Metadata.empty) + + override def toString: String = toMetadata(withType = true).toString +} + +/** Trait for ML attribute factories. */ +private[attribute] trait AttributeFactory { + + /** + * Creates an [[Attribute]] from a [[Metadata]] instance. + */ + private[attribute] def fromMetadata(metadata: Metadata): Attribute + + /** + * Creates an [[Attribute]] from a [[StructField]] instance. + */ + def fromStructField(field: StructField): Attribute = { + require(field.dataType == DoubleType) + fromMetadata(field.metadata.getMetadata(AttributeKeys.ML_ATTR)).withName(field.name) + } +} + +object Attribute extends AttributeFactory { + + private[attribute] override def fromMetadata(metadata: Metadata): Attribute = { + import org.apache.spark.ml.attribute.AttributeKeys._ + val attrType = if (metadata.contains(TYPE)) { + metadata.getString(TYPE) + } else { + AttributeType.Numeric.name + } + getFactory(attrType).fromMetadata(metadata) + } + + /** Gets the attribute factory given the attribute type name. */ + private def getFactory(attrType: String): AttributeFactory = { + if (attrType == AttributeType.Numeric.name) { + NumericAttribute + } else if (attrType == AttributeType.Nominal.name) { + NominalAttribute + } else if (attrType == AttributeType.Binary.name) { + BinaryAttribute + } else { + throw new IllegalArgumentException(s"Cannot recognize type $attrType.") + } + } +} + + +/** + * A numeric attribute with optional summary statistics. + * @param name optional name + * @param index optional index + * @param min optional min value + * @param max optional max value + * @param std optional standard deviation + * @param sparsity optional sparsity (ratio of zeros) + */ +class NumericAttribute private[ml] ( + override val name: Option[String] = None, + override val index: Option[Int] = None, + val min: Option[Double] = None, + val max: Option[Double] = None, + val std: Option[Double] = None, + val sparsity: Option[Double] = None) extends Attribute { + + std.foreach { s => + require(s >= 0.0, s"Standard deviation cannot be negative but got $s.") + } + sparsity.foreach { s => + require(s >= 0.0 && s <= 1.0, s"Sparsity must be in [0, 1] but got $s.") + } + + override def attrType: AttributeType = AttributeType.Numeric + + override def withName(name: String): NumericAttribute = copy(name = Some(name)) + override def withoutName: NumericAttribute = copy(name = None) + + override def withIndex(index: Int): NumericAttribute = copy(index = Some(index)) + override def withoutIndex: NumericAttribute = copy(index = None) + + /** Copy with a new min value. */ + def withMin(min: Double): NumericAttribute = copy(min = Some(min)) + + /** Copy without the min value. */ + def withoutMin: NumericAttribute = copy(min = None) + + + /** Copy with a new max value. */ + def withMax(max: Double): NumericAttribute = copy(max = Some(max)) + + /** Copy without the max value. */ + def withoutMax: NumericAttribute = copy(max = None) + + /** Copy with a new standard deviation. */ + def withStd(std: Double): NumericAttribute = copy(std = Some(std)) + + /** Copy without the standard deviation. */ + def withoutStd: NumericAttribute = copy(std = None) + + /** Copy with a new sparsity. */ + def withSparsity(sparsity: Double): NumericAttribute = copy(sparsity = Some(sparsity)) + + /** Copy without the sparsity. */ + def withoutSparsity: NumericAttribute = copy(sparsity = None) + + /** Copy without summary statistics. */ + def withoutSummary: NumericAttribute = copy(min = None, max = None, std = None, sparsity = None) + + override def isNumeric: Boolean = true + + override def isNominal: Boolean = false + + /** Convert this attribute to metadata. */ + private[attribute] override def toMetadata(withType: Boolean): Metadata = { + import org.apache.spark.ml.attribute.AttributeKeys._ + val bldr = new MetadataBuilder() + if (withType) bldr.putString(TYPE, attrType.name) + name.foreach(bldr.putString(NAME, _)) + index.foreach(bldr.putLong(INDEX, _)) + min.foreach(bldr.putDouble(MIN, _)) + max.foreach(bldr.putDouble(MAX, _)) + std.foreach(bldr.putDouble(STD, _)) + sparsity.foreach(bldr.putDouble(SPARSITY, _)) + bldr.build() + } + + /** Creates a copy of this attribute with optional changes. */ + private def copy( + name: Option[String] = name, + index: Option[Int] = index, + min: Option[Double] = min, + max: Option[Double] = max, + std: Option[Double] = std, + sparsity: Option[Double] = sparsity): NumericAttribute = { + new NumericAttribute(name, index, min, max, std, sparsity) + } + + override def equals(other: Any): Boolean = { + other match { + case o: NumericAttribute => + (name == o.name) && + (index == o.index) && + (min == o.min) && + (max == o.max) && + (std == o.std) && + (sparsity == o.sparsity) + case _ => + false + } + } + + override def hashCode: Int = { + var sum = 17 + sum = 37 * sum + name.hashCode + sum = 37 * sum + index.hashCode + sum = 37 * sum + min.hashCode + sum = 37 * sum + max.hashCode + sum = 37 * sum + std.hashCode + sum = 37 * sum + sparsity.hashCode + sum + } +} + +/** + * Factory methods for numeric attributes. + */ +object NumericAttribute extends AttributeFactory { + + /** The default numeric attribute. */ + val defaultAttr: NumericAttribute = new NumericAttribute + + private[attribute] override def fromMetadata(metadata: Metadata): NumericAttribute = { + import org.apache.spark.ml.attribute.AttributeKeys._ + val name = if (metadata.contains(NAME)) Some(metadata.getString(NAME)) else None + val index = if (metadata.contains(INDEX)) Some(metadata.getLong(INDEX).toInt) else None + val min = if (metadata.contains(MIN)) Some(metadata.getDouble(MIN)) else None + val max = if (metadata.contains(MAX)) Some(metadata.getDouble(MAX)) else None + val std = if (metadata.contains(STD)) Some(metadata.getDouble(STD)) else None + val sparsity = if (metadata.contains(SPARSITY)) Some(metadata.getDouble(SPARSITY)) else None + new NumericAttribute(name, index, min, max, std, sparsity) + } +} + +/** + * A nominal attribute. + * @param name optional name + * @param index optional index + * @param isOrdinal whether this attribute is ordinal (optional) + * @param numValues optional number of values. At most one of `numValues` and `values` can be + * defined. + * @param values optional values. At most one of `numValues` and `values` can be defined. + */ +class NominalAttribute private[ml] ( + override val name: Option[String] = None, + override val index: Option[Int] = None, + val isOrdinal: Option[Boolean] = None, + val numValues: Option[Int] = None, + val values: Option[Array[String]] = None) extends Attribute { + + numValues.foreach { n => + require(n >= 0, s"numValues cannot be negative but got $n.") + } + require(!(numValues.isDefined && values.isDefined), + "Cannot have both numValues and values defined.") + + override def attrType: AttributeType = AttributeType.Nominal + + override def isNumeric: Boolean = false + + override def isNominal: Boolean = true + + private lazy val valueToIndex: Map[String, Int] = { + values.map(_.zipWithIndex.toMap).getOrElse(Map.empty) + } + + /** Index of a specific value. */ + def indexOf(value: String): Int = { + valueToIndex(value) + } + + /** Tests whether this attribute contains a specific value. */ + def hasValue(value: String): Boolean = valueToIndex.contains(value) + + /** Gets a value given its index. */ + def getValue(index: Int): String = values.get(index) + + override def withName(name: String): NominalAttribute = copy(name = Some(name)) + override def withoutName: NominalAttribute = copy(name = None) + + override def withIndex(index: Int): NominalAttribute = copy(index = Some(index)) + override def withoutIndex: NominalAttribute = copy(index = None) + + /** Copy with new values and empty `numValues`. */ + def withValues(values: Array[String]): NominalAttribute = { + copy(numValues = None, values = Some(values)) + } + + /** Copy with new values and empty `numValues`. */ + @varargs + def withValues(first: String, others: String*): NominalAttribute = { + copy(numValues = None, values = Some((first +: others).toArray)) + } + + /** Copy without the values. */ + def withoutValues: NominalAttribute = { + copy(values = None) + } + + /** Copy with a new `numValues` and empty `values`. */ + def withNumValues(numValues: Int): NominalAttribute = { + copy(numValues = Some(numValues), values = None) + } + + /** Copy without the `numValues`. */ + def withoutNumValues: NominalAttribute = copy(numValues = None) + + /** Creates a copy of this attribute with optional changes. */ + private def copy( + name: Option[String] = name, + index: Option[Int] = index, + isOrdinal: Option[Boolean] = isOrdinal, + numValues: Option[Int] = numValues, + values: Option[Array[String]] = values): NominalAttribute = { + new NominalAttribute(name, index, isOrdinal, numValues, values) + } + + private[attribute] override def toMetadata(withType: Boolean): Metadata = { + import org.apache.spark.ml.attribute.AttributeKeys._ + val bldr = new MetadataBuilder() + if (withType) bldr.putString(TYPE, attrType.name) + name.foreach(bldr.putString(NAME, _)) + index.foreach(bldr.putLong(INDEX, _)) + isOrdinal.foreach(bldr.putBoolean(ORDINAL, _)) + numValues.foreach(bldr.putLong(NUM_VALUES, _)) + values.foreach(v => bldr.putStringArray(VALUES, v)) + bldr.build() + } + + override def equals(other: Any): Boolean = { + other match { + case o: NominalAttribute => + (name == o.name) && + (index == o.index) && + (isOrdinal == o.isOrdinal) && + (numValues == o.numValues) && + (values.map(_.toSeq) == o.values.map(_.toSeq)) + case _ => + false + } + } + + override def hashCode: Int = { + var sum = 17 + sum = 37 * sum + name.hashCode + sum = 37 * sum + index.hashCode + sum = 37 * sum + isOrdinal.hashCode + sum = 37 * sum + numValues.hashCode + sum = 37 * sum + values.map(_.toSeq).hashCode + sum + } +} + +/** Factory methods for nominal attributes. */ +object NominalAttribute extends AttributeFactory { + + /** The default nominal attribute. */ + final val defaultAttr: NominalAttribute = new NominalAttribute + + private[attribute] override def fromMetadata(metadata: Metadata): NominalAttribute = { + import org.apache.spark.ml.attribute.AttributeKeys._ + val name = if (metadata.contains(NAME)) Some(metadata.getString(NAME)) else None + val index = if (metadata.contains(INDEX)) Some(metadata.getLong(INDEX).toInt) else None + val isOrdinal = if (metadata.contains(ORDINAL)) Some(metadata.getBoolean(ORDINAL)) else None + val numValues = + if (metadata.contains(NUM_VALUES)) Some(metadata.getLong(NUM_VALUES).toInt) else None + val values = + if (metadata.contains(VALUES)) Some(metadata.getStringArray(VALUES)) else None + new NominalAttribute(name, index, isOrdinal, numValues, values) + } +} + +/** + * A binary attribute. + * @param name optional name + * @param index optional index + * @param values optionla values. If set, its size must be 2. + */ +class BinaryAttribute private[ml] ( + override val name: Option[String] = None, + override val index: Option[Int] = None, + val values: Option[Array[String]] = None) + extends Attribute { + + values.foreach { v => + require(v.length == 2, s"Number of values must be 2 for a binary attribute but got ${v.toSeq}.") + } + + override def attrType: AttributeType = AttributeType.Binary + + override def isNumeric: Boolean = true + + override def isNominal: Boolean = true + + override def withName(name: String): BinaryAttribute = copy(name = Some(name)) + override def withoutName: BinaryAttribute = copy(name = None) + + override def withIndex(index: Int): BinaryAttribute = copy(index = Some(index)) + override def withoutIndex: BinaryAttribute = copy(index = None) + + /** + * Copy with new values. + * @param negative name for negative + * @param positive name for positive + */ + def withValues(negative: String, positive: String): BinaryAttribute = + copy(values = Some(Array(negative, positive))) + + /** Copy without the values. */ + def withoutValues: BinaryAttribute = copy(values = None) + + /** Creates a copy of this attribute with optional changes. */ + private def copy( + name: Option[String] = name, + index: Option[Int] = index, + values: Option[Array[String]] = values): BinaryAttribute = { + new BinaryAttribute(name, index, values) + } + + private[attribute] override def toMetadata(withType: Boolean): Metadata = { + import org.apache.spark.ml.attribute.AttributeKeys._ + val bldr = new MetadataBuilder + if (withType) bldr.putString(TYPE, attrType.name) + name.foreach(bldr.putString(NAME, _)) + index.foreach(bldr.putLong(INDEX, _)) + values.foreach(v => bldr.putStringArray(VALUES, v)) + bldr.build() + } + + override def equals(other: Any): Boolean = { + other match { + case o: BinaryAttribute => + (name == o.name) && + (index == o.index) && + (values.map(_.toSeq) == o.values.map(_.toSeq)) + case _ => + false + } + } + + override def hashCode: Int = { + var sum = 17 + sum = 37 * sum + name.hashCode + sum = 37 * sum + index.hashCode + sum = 37 * sum + values.map(_.toSeq).hashCode + sum + } +} + +/** Factory methods for binary attributes. */ +object BinaryAttribute extends AttributeFactory { + + /** The default binary attribute. */ + final val defaultAttr: BinaryAttribute = new BinaryAttribute + + private[attribute] override def fromMetadata(metadata: Metadata): BinaryAttribute = { + import org.apache.spark.ml.attribute.AttributeKeys._ + val name = if (metadata.contains(NAME)) Some(metadata.getString(NAME)) else None + val index = if (metadata.contains(INDEX)) Some(metadata.getLong(INDEX).toInt) else None + val values = + if (metadata.contains(VALUES)) Some(metadata.getStringArray(VALUES)) else None + new BinaryAttribute(name, index, values) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/package-info.java b/mllib/src/main/scala/org/apache/spark/ml/attribute/package-info.java new file mode 100644 index 0000000000000..e3474f3c1d3ff --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/package-info.java @@ -0,0 +1,41 @@ +/* + * 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. + */ + +// The content here should be in sync with `package.scala`. + +/** + *

ML attributes

+ * + * The ML pipeline API uses {@link org.apache.spark.sql.DataFrame}s as ML datasets. + * Each dataset consists of typed columns, e.g., string, double, vector, etc. + * However, knowing only the column type may not be sufficient to handle the data properly. + * For instance, a double column with values 0.0, 1.0, 2.0, ... may represent some label indices, + * which cannot be treated as numeric values in ML algorithms, and, for another instance, we may + * want to know the names and types of features stored in a vector column. + * ML attributes are used to provide additional information to describe columns in a dataset. + * + *

ML columns

+ * + * A column with ML attributes attached is called an ML column. + * The data in ML columns are stored as double values, i.e., an ML column is either a scalar column + * of double values or a vector column. + * Columns of other types must be encoded into ML columns using transformers. + * We use {@link org.apache.spark.ml.attribute.Attribute} to describe a scalar ML column, and + * {@link org.apache.spark.ml.attribute.AttributeGroup} to describe a vector ML column. + * ML attributes are stored in the metadata field of the column schema. + */ +package org.apache.spark.ml.attribute; diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/package.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/package.scala new file mode 100644 index 0000000000000..7ac21d7d563f2 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/package.scala @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml + +import org.apache.spark.sql.DataFrame +import org.apache.spark.ml.attribute.{Attribute, AttributeGroup} + +/** + * ==ML attributes== + * + * The ML pipeline API uses [[DataFrame]]s as ML datasets. + * Each dataset consists of typed columns, e.g., string, double, vector, etc. + * However, knowing only the column type may not be sufficient to handle the data properly. + * For instance, a double column with values 0.0, 1.0, 2.0, ... may represent some label indices, + * which cannot be treated as numeric values in ML algorithms, and, for another instance, we may + * want to know the names and types of features stored in a vector column. + * ML attributes are used to provide additional information to describe columns in a dataset. + * + * ===ML columns=== + * + * A column with ML attributes attached is called an ML column. + * The data in ML columns are stored as double values, i.e., an ML column is either a scalar column + * of double values or a vector column. + * Columns of other types must be encoded into ML columns using transformers. + * We use [[Attribute]] to describe a scalar ML column, and [[AttributeGroup]] to describe a vector + * ML column. + * ML attributes are stored in the metadata field of the column schema. + */ +package object attribute diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 7bb69df65362b..e3515ee81af3d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -26,7 +26,6 @@ import scala.util.hashing.byteswap64 import com.github.fommil.netlib.BLAS.{getInstance => blas} import com.github.fommil.netlib.LAPACK.{getInstance => lapack} -import org.jblas.DoubleMatrix import org.netlib.util.intW import org.apache.spark.{Logging, Partitioner} @@ -361,14 +360,14 @@ object ALS extends Logging { private[recommendation] class NNLSSolver extends LeastSquaresNESolver { private var rank: Int = -1 private var workspace: NNLS.Workspace = _ - private var ata: DoubleMatrix = _ + private var ata: Array[Double] = _ private var initialized: Boolean = false private def initialize(rank: Int): Unit = { if (!initialized) { this.rank = rank workspace = NNLS.createWorkspace(rank) - ata = new DoubleMatrix(rank, rank) + ata = new Array[Double](rank * rank) initialized = true } else { require(this.rank == rank) @@ -385,7 +384,7 @@ object ALS extends Logging { val rank = ne.k initialize(rank) fillAtA(ne.ata, lambda * ne.n) - val x = NNLS.solve(ata, new DoubleMatrix(rank, 1, ne.atb: _*), workspace) + val x = NNLS.solve(ata, ne.atb, workspace) ne.reset() x.map(x => x.toFloat) } @@ -398,17 +397,16 @@ object ALS extends Logging { var i = 0 var pos = 0 var a = 0.0 - val data = ata.data while (i < rank) { var j = 0 while (j <= i) { a = triAtA(pos) - data(i * rank + j) = a - data(j * rank + i) = a + ata(i * rank + j) = a + ata(j * rank + i) = a pos += 1 j += 1 } - data(i * rank + i) += lambda + ata(i * rank + i) += lambda i += 1 } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index b787667b018e6..e7c3599ff619c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -163,6 +163,10 @@ class LogisticRegressionModel ( } override protected def formatVersion: String = "1.0" + + override def toString: String = { + s"${super.toString}, numClasses = ${numClasses}, threshold = ${threshold.get}" + } } object LogisticRegressionModel extends Loader[LogisticRegressionModel] { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index b11fd4f128c56..2ebc7fa5d4234 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -166,6 +166,9 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with this } + /** Get the smoothing parameter. Default: 1.0. */ + def getLambda: Double = lambda + /** * Run the algorithm with the configured parameters on an input RDD of LabeledPoint entries. * diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index cfc7f868a02f0..52fb62dcff1b4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -86,6 +86,10 @@ class SVMModel ( } override protected def formatVersion: String = "1.0" + + override def toString: String = { + s"${super.toString}, numClasses = 2, threshold = ${threshold.get}" + } } object SVMModel extends Loader[SVMModel] { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 11633e8242313..e41f941fd2c2c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -52,18 +52,33 @@ class KMeans private ( */ def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4, Utils.random.nextLong()) + /** + * Number of clusters to create (k). + */ + def getK: Int = k + /** Set the number of clusters to create (k). Default: 2. */ def setK(k: Int): this.type = { this.k = k this } + /** + * Maximum number of iterations to run. + */ + def getMaxIterations: Int = maxIterations + /** Set maximum number of iterations to run. Default: 20. */ def setMaxIterations(maxIterations: Int): this.type = { this.maxIterations = maxIterations this } + /** + * The initialization algorithm. This can be either "random" or "k-means||". + */ + def getInitializationMode: String = initializationMode + /** * Set the initialization algorithm. This can be either "random" to choose random points as * initial cluster centers, or "k-means||" to use a parallel variant of k-means++ @@ -77,6 +92,13 @@ class KMeans private ( this } + /** + * :: Experimental :: + * Number of runs of the algorithm to execute in parallel. + */ + @Experimental + def getRuns: Int = runs + /** * :: Experimental :: * Set the number of runs of the algorithm to execute in parallel. We initialize the algorithm @@ -92,6 +114,11 @@ class KMeans private ( this } + /** + * Number of steps for the k-means|| initialization mode + */ + def getInitializationSteps: Int = initializationSteps + /** * Set the number of steps for the k-means|| initialization mode. This is an advanced * setting -- the default of 5 is almost always enough. Default: 5. @@ -104,6 +131,11 @@ class KMeans private ( this } + /** + * The distance threshold within which we've consider centers to have converged. + */ + def getEpsilon: Double = epsilon + /** * Set the distance threshold within which we've consider centers to have converged. * If all centers move less than this Euclidean distance, we stop iterating one run. @@ -113,6 +145,11 @@ class KMeans private ( this } + /** + * The random seed for cluster initialization. + */ + def getSeed: Long = seed + /** Set the random seed for cluster initialization. */ def setSeed(seed: Long): this.type = { this.seed = seed diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 707da537d238f..e4e411a3c8b42 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -17,6 +17,8 @@ package org.apache.spark.mllib.clustering +import scala.collection.JavaConverters._ + import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ @@ -34,6 +36,9 @@ import org.apache.spark.sql.Row */ class KMeansModel (val clusterCenters: Array[Vector]) extends Saveable with Serializable { + /** A Java-friendly constructor that takes an Iterable of Vectors. */ + def this(centers: java.lang.Iterable[Vector]) = this(centers.asScala.toArray) + /** Total number of clusters. */ def k: Int = clusterCenters.length diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index d5e4f4ccbff10..ef6eccd90711a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -60,6 +60,8 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) /** * Set the convergence tolerance of iterations for L-BFGS. Default 1E-4. * Smaller value will lead to higher accuracy with the cost of more iterations. + * This value must be nonnegative. Lower convergence values are less tolerant + * and therefore generally cause more iterations to be run. */ def setConvergenceTol(tolerance: Double): this.type = { this.convergenceTol = tolerance @@ -142,7 +144,9 @@ object LBFGS extends Logging { * one single data example) * @param updater - Updater function to actually perform a gradient step in a given direction. * @param numCorrections - The number of corrections used in the L-BFGS update. - * @param convergenceTol - The convergence tolerance of iterations for L-BFGS + * @param convergenceTol - The convergence tolerance of iterations for L-BFGS which is must be + * nonnegative. Lower values are less tolerant and therefore generally + * cause more iterations to be run. * @param maxNumIterations - Maximal number of iterations that L-BFGS can be run. * @param regParam - Regularization parameter * diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala index ccd93b318bc23..4766f7708295d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala @@ -17,7 +17,9 @@ package org.apache.spark.mllib.optimization -import org.jblas.{DoubleMatrix, SimpleBlas} +import java.{util => ju} + +import com.github.fommil.netlib.BLAS.{getInstance => blas} /** * Object used to solve nonnegative least squares problems using a modified @@ -25,20 +27,20 @@ import org.jblas.{DoubleMatrix, SimpleBlas} */ private[spark] object NNLS { class Workspace(val n: Int) { - val scratch = new DoubleMatrix(n, 1) - val grad = new DoubleMatrix(n, 1) - val x = new DoubleMatrix(n, 1) - val dir = new DoubleMatrix(n, 1) - val lastDir = new DoubleMatrix(n, 1) - val res = new DoubleMatrix(n, 1) - - def wipe() { - scratch.fill(0.0) - grad.fill(0.0) - x.fill(0.0) - dir.fill(0.0) - lastDir.fill(0.0) - res.fill(0.0) + val scratch = new Array[Double](n) + val grad = new Array[Double](n) + val x = new Array[Double](n) + val dir = new Array[Double](n) + val lastDir = new Array[Double](n) + val res = new Array[Double](n) + + def wipe(): Unit = { + ju.Arrays.fill(scratch, 0.0) + ju.Arrays.fill(grad, 0.0) + ju.Arrays.fill(x, 0.0) + ju.Arrays.fill(dir, 0.0) + ju.Arrays.fill(lastDir, 0.0) + ju.Arrays.fill(res, 0.0) } } @@ -60,18 +62,18 @@ private[spark] object NNLS { * direction, however, while this method only uses a conjugate gradient direction if the last * iteration did not cause a previously-inactive constraint to become active. */ - def solve(ata: DoubleMatrix, atb: DoubleMatrix, ws: Workspace): Array[Double] = { + def solve(ata: Array[Double], atb: Array[Double], ws: Workspace): Array[Double] = { ws.wipe() - val n = atb.rows + val n = atb.length val scratch = ws.scratch // find the optimal unconstrained step - def steplen(dir: DoubleMatrix, res: DoubleMatrix): Double = { - val top = SimpleBlas.dot(dir, res) - SimpleBlas.gemv(1.0, ata, dir, 0.0, scratch) + def steplen(dir: Array[Double], res: Array[Double]): Double = { + val top = blas.ddot(n, dir, 1, res, 1) + blas.dgemv("N", n, n, 1.0, ata, n, dir, 1, 0.0, scratch, 1) // Push the denominator upward very slightly to avoid infinities and silliness - top / (SimpleBlas.dot(scratch, dir) + 1e-20) + top / (blas.ddot(n, scratch, 1, dir, 1) + 1e-20) } // stopping condition @@ -96,52 +98,52 @@ private[spark] object NNLS { var i = 0 while (iterno < iterMax) { // find the residual - SimpleBlas.gemv(1.0, ata, x, 0.0, res) - SimpleBlas.axpy(-1.0, atb, res) - SimpleBlas.copy(res, grad) + blas.dgemv("N", n, n, 1.0, ata, n, x, 1, 0.0, res, 1) + blas.daxpy(n, -1.0, atb, 1, res, 1) + blas.dcopy(n, res, 1, grad, 1) // project the gradient i = 0 while (i < n) { - if (grad.data(i) > 0.0 && x.data(i) == 0.0) { - grad.data(i) = 0.0 + if (grad(i) > 0.0 && x(i) == 0.0) { + grad(i) = 0.0 } i = i + 1 } - val ngrad = SimpleBlas.dot(grad, grad) + val ngrad = blas.ddot(n, grad, 1, grad, 1) - SimpleBlas.copy(grad, dir) + blas.dcopy(n, grad, 1, dir, 1) // use a CG direction under certain conditions var step = steplen(grad, res) var ndir = 0.0 - val nx = SimpleBlas.dot(x, x) + val nx = blas.ddot(n, x, 1, x, 1) if (iterno > lastWall + 1) { val alpha = ngrad / lastNorm - SimpleBlas.axpy(alpha, lastDir, dir) + blas.daxpy(n, alpha, lastDir, 1, dir, 1) val dstep = steplen(dir, res) - ndir = SimpleBlas.dot(dir, dir) + ndir = blas.ddot(n, dir, 1, dir, 1) if (stop(dstep, ndir, nx)) { // reject the CG step if it could lead to premature termination - SimpleBlas.copy(grad, dir) - ndir = SimpleBlas.dot(dir, dir) + blas.dcopy(n, grad, 1, dir, 1) + ndir = blas.ddot(n, dir, 1, dir, 1) } else { step = dstep } } else { - ndir = SimpleBlas.dot(dir, dir) + ndir = blas.ddot(n, dir, 1, dir, 1) } // terminate? if (stop(step, ndir, nx)) { - return x.data.clone + return x.clone } // don't run through the walls i = 0 while (i < n) { - if (step * dir.data(i) > x.data(i)) { - step = x.data(i) / dir.data(i) + if (step * dir(i) > x(i)) { + step = x(i) / dir(i) } i = i + 1 } @@ -149,19 +151,19 @@ private[spark] object NNLS { // take the step i = 0 while (i < n) { - if (step * dir.data(i) > x.data(i) * (1 - 1e-14)) { - x.data(i) = 0 + if (step * dir(i) > x(i) * (1 - 1e-14)) { + x(i) = 0 lastWall = iterno } else { - x.data(i) -= step * dir.data(i) + x(i) -= step * dir(i) } i = i + 1 } iterno = iterno + 1 - SimpleBlas.copy(dir, lastDir) + blas.dcopy(n, dir, 1, lastDir, 1) lastNorm = ngrad } - x.data.clone + x.clone } } 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 c399496568bfb..36cbf060d9998 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 @@ -21,10 +21,10 @@ import java.io.IOException import java.lang.{Integer => JavaInteger} import org.apache.hadoop.fs.Path -import org.jblas.DoubleMatrix import org.json4s._ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ +import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.{Logging, SparkContext} import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} @@ -70,9 +70,9 @@ class MatrixFactorizationModel( /** Predict the rating of one user for one product. */ def predict(user: Int, product: Int): Double = { - val userVector = new DoubleMatrix(userFeatures.lookup(user).head) - val productVector = new DoubleMatrix(productFeatures.lookup(product).head) - userVector.dot(productVector) + val userVector = userFeatures.lookup(user).head + val productVector = productFeatures.lookup(product).head + blas.ddot(userVector.length, userVector, 1, productVector, 1) } /** @@ -89,9 +89,7 @@ class MatrixFactorizationModel( } users.join(productFeatures).map { case (product, ((user, uFeatures), pFeatures)) => - val userVector = new DoubleMatrix(uFeatures) - val productVector = new DoubleMatrix(pFeatures) - Rating(user, product, userVector.dot(productVector)) + Rating(user, product, blas.ddot(uFeatures.length, uFeatures, 1, pFeatures, 1)) } } @@ -143,9 +141,8 @@ class MatrixFactorizationModel( recommendToFeatures: Array[Double], recommendableFeatures: RDD[(Int, Array[Double])], num: Int): Array[(Int, Double)] = { - val recommendToVector = new DoubleMatrix(recommendToFeatures) val scored = recommendableFeatures.map { case (id,features) => - (id, recommendToVector.dot(new DoubleMatrix(features))) + (id, blas.ddot(features.length, recommendToFeatures, 1, features, 1)) } scored.top(num)(Ordering.by(_._2)) } @@ -199,12 +196,12 @@ object MatrixFactorizationModel extends Loader[MatrixFactorizationModel] { assert(formatVersion == thisFormatVersion) val rank = (metadata \ "rank").extract[Int] val userFeatures = sqlContext.parquetFile(userPath(path)) - .map { case Row(id: Int, features: Seq[Double]) => - (id, features.toArray) + .map { case Row(id: Int, features: Seq[_]) => + (id, features.asInstanceOf[Seq[Double]].toArray) } val productFeatures = sqlContext.parquetFile(productPath(path)) - .map { case Row(id: Int, features: Seq[Double]) => - (id, features.toArray) + .map { case Row(id: Int, features: Seq[_]) => + (id, features.asInstanceOf[Seq[Double]].toArray) } new MatrixFactorizationModel(rank, userFeatures, productFeatures) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 7c66e8cdebdbe..45b9ebb4cc0d6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -76,7 +76,12 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double predictPoint(testData, weights, intercept) } - override def toString() = "(weights=%s, intercept=%s)".format(weights, intercept) + /** + * Print a summary of the model. + */ + override def toString: String = { + s"${this.getClass.getName}: intercept = ${intercept}, numFeatures = ${weights.size}" + } } /** @@ -123,6 +128,11 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] */ private var useFeatureScaling = false + /** + * The dimension of training features. + */ + def getNumFeatures: Int = this.numFeatures + /** * The dimension of training features. */ @@ -141,6 +151,11 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] */ protected def createModel(weights: Vector, intercept: Double): M + /** + * Get if the algorithm uses addIntercept + */ + def isAddIntercept: Boolean = this.addIntercept + /** * Set if the algorithm should add an intercept. Default false. * We set the default to false because adding the intercept will cause memory allocation. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 97f54aa62d31c..c9d33787b0bb5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.util import scala.collection.JavaConversions._ import scala.util.Random -import org.jblas.DoubleMatrix +import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext @@ -72,11 +72,10 @@ object LinearDataGenerator { eps: Double = 0.1): Seq[LabeledPoint] = { val rnd = new Random(seed) - val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) val x = Array.fill[Array[Double]](nPoints)( Array.fill[Double](weights.length)(2 * rnd.nextDouble - 1.0)) val y = x.map { xi => - new DoubleMatrix(1, xi.length, xi: _*).dot(weightsMat) + intercept + eps * rnd.nextGaussian() + blas.ddot(weights.length, xi, 1, weights, 1) + intercept + eps * rnd.nextGaussian() } y.zip(x).map(p => LabeledPoint(p._1, Vectors.dense(p._2))) } @@ -100,9 +99,9 @@ object LinearDataGenerator { eps: Double, nparts: Int = 2, intercept: Double = 0.0) : RDD[LabeledPoint] = { - org.jblas.util.Random.seed(42) + val random = new Random(42) // Random values distributed uniformly in [-0.5, 0.5] - val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) + val w = Array.fill(nfeatures)(random.nextDouble() - 0.5) val data: RDD[LabeledPoint] = sc.parallelize(0 until nparts, nparts).flatMap { p => val seed = 42 + p diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index b76fbe89c3681..0c5b4f9d04a74 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -17,13 +17,14 @@ package org.apache.spark.mllib.util +import java.{util => ju} + import scala.language.postfixOps import scala.util.Random -import org.jblas.DoubleMatrix - -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.linalg.{BLAS, DenseMatrix} import org.apache.spark.rdd.RDD /** @@ -72,24 +73,25 @@ object MFDataGenerator { val sc = new SparkContext(sparkMaster, "MFDataGenerator") - val A = DoubleMatrix.randn(m, rank) - val B = DoubleMatrix.randn(rank, n) - val z = 1 / scala.math.sqrt(scala.math.sqrt(rank)) - A.mmuli(z) - B.mmuli(z) - val fullData = A.mmul(B) + val random = new ju.Random(42L) + + val A = DenseMatrix.randn(m, rank, random) + val B = DenseMatrix.randn(rank, n, random) + val z = 1 / math.sqrt(rank) + val fullData = DenseMatrix.zeros(m, n) + BLAS.gemm(z, A, B, 1.0, fullData) val df = rank * (m + n - rank) val sampSize = scala.math.min(scala.math.round(trainSampFact * df), scala.math.round(.99 * m * n)).toInt val rand = new Random() val mn = m * n - val shuffled = rand.shuffle(1 to mn toList) + val shuffled = rand.shuffle((0 until mn).toList) val omega = shuffled.slice(0, sampSize) val ordered = omega.sortWith(_ < _).toArray val trainData: RDD[(Int, Int, Double)] = sc.parallelize(ordered) - .map(x => (fullData.indexRows(x - 1), fullData.indexColumns(x - 1), fullData.get(x - 1))) + .map(x => (x % m, x / m, fullData.values(x))) // optionally add gaussian noise if (noise) { @@ -105,7 +107,7 @@ object MFDataGenerator { val testOmega = shuffled.slice(sampSize, sampSize + testSampSize) val testOrdered = testOmega.sortWith(_ < _).toArray val testData: RDD[(Int, Int, Double)] = sc.parallelize(testOrdered) - .map(x => (fullData.indexRows(x - 1), fullData.indexColumns(x - 1), fullData.get(x - 1))) + .map(x => (x % m, x / m, fullData.values(x))) testData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index 7db97e6bac688..a8e30cc9d730c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.util import scala.util.Random -import org.jblas.DoubleMatrix +import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext @@ -51,8 +51,7 @@ object SVMDataGenerator { val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) - val trueWeights = new DoubleMatrix(1, nfeatures + 1, - Array.fill[Double](nfeatures + 1)(globalRnd.nextGaussian()):_*) + val trueWeights = Array.fill[Double](nfeatures + 1)(globalRnd.nextGaussian()) val data: RDD[LabeledPoint] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) @@ -60,7 +59,7 @@ object SVMDataGenerator { val x = Array.fill[Double](nfeatures) { rnd.nextDouble() * 2.0 - 1.0 } - val yD = new DoubleMatrix(1, x.length, x: _*).dot(trueWeights) + rnd.nextGaussian() * 0.1 + val yD = blas.ddot(trueWeights.length, x, 1, trueWeights, 1) + rnd.nextGaussian() * 0.1 val y = if (yD < 0) 0.0 else 1.0 LabeledPoint(y, Vectors.dense(x)) } diff --git a/mllib/src/test/java/org/apache/spark/ml/attribute/JavaAttributeGroupSuite.java b/mllib/src/test/java/org/apache/spark/ml/attribute/JavaAttributeGroupSuite.java new file mode 100644 index 0000000000000..38eb58673ad53 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/attribute/JavaAttributeGroupSuite.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.attribute; + +import org.junit.Assert; +import org.junit.Test; + +public class JavaAttributeGroupSuite { + + @Test + public void testAttributeGroup() { + Attribute[] attrs = new Attribute[]{ + NumericAttribute.defaultAttr(), + NominalAttribute.defaultAttr(), + BinaryAttribute.defaultAttr().withIndex(0), + NumericAttribute.defaultAttr().withName("age").withSparsity(0.8), + NominalAttribute.defaultAttr().withName("size").withValues("small", "medium", "large"), + BinaryAttribute.defaultAttr().withName("clicked").withValues("no", "yes"), + NumericAttribute.defaultAttr(), + NumericAttribute.defaultAttr() + }; + AttributeGroup group = new AttributeGroup("user", attrs); + Assert.assertEquals(8, group.size()); + Assert.assertEquals("user", group.name()); + Assert.assertEquals(NumericAttribute.defaultAttr().withIndex(0), group.getAttr(0)); + Assert.assertEquals(3, group.indexOf("age")); + Assert.assertFalse(group.hasAttr("abc")); + Assert.assertEquals(group, AttributeGroup.fromStructField(group.toStructField())); + } +} diff --git a/mllib/src/test/java/org/apache/spark/ml/attribute/JavaAttributeSuite.java b/mllib/src/test/java/org/apache/spark/ml/attribute/JavaAttributeSuite.java new file mode 100644 index 0000000000000..b74bbed231434 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/attribute/JavaAttributeSuite.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.attribute; + +import org.junit.Test; +import org.junit.Assert; + +public class JavaAttributeSuite { + + @Test + public void testAttributeType() { + AttributeType numericType = AttributeType.Numeric(); + AttributeType nominalType = AttributeType.Nominal(); + AttributeType binaryType = AttributeType.Binary(); + Assert.assertEquals(numericType, NumericAttribute.defaultAttr().attrType()); + Assert.assertEquals(nominalType, NominalAttribute.defaultAttr().attrType()); + Assert.assertEquals(binaryType, BinaryAttribute.defaultAttr().attrType()); + } + + @Test + public void testNumericAttribute() { + NumericAttribute attr = NumericAttribute.defaultAttr() + .withName("age").withIndex(0).withMin(0.0).withMax(1.0).withStd(0.5).withSparsity(0.4); + Assert.assertEquals(attr.withoutIndex(), Attribute.fromStructField(attr.toStructField())); + } + + @Test + public void testNominalAttribute() { + NominalAttribute attr = NominalAttribute.defaultAttr() + .withName("size").withIndex(1).withValues("small", "medium", "large"); + Assert.assertEquals(attr.withoutIndex(), Attribute.fromStructField(attr.toStructField())); + } + + @Test + public void testBinaryAttribute() { + BinaryAttribute attr = BinaryAttribute.defaultAttr() + .withName("clicked").withIndex(2).withValues("no", "yes"); + Assert.assertEquals(attr.withoutIndex(), Attribute.fromStructField(attr.toStructField())); + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala new file mode 100644 index 0000000000000..3fb6e2ec46468 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.attribute + +import org.scalatest.FunSuite + +class AttributeGroupSuite extends FunSuite { + + test("attribute group") { + val attrs = Array( + NumericAttribute.defaultAttr, + NominalAttribute.defaultAttr, + BinaryAttribute.defaultAttr.withIndex(0), + NumericAttribute.defaultAttr.withName("age").withSparsity(0.8), + NominalAttribute.defaultAttr.withName("size").withValues("small", "medium", "large"), + BinaryAttribute.defaultAttr.withName("clicked").withValues("no", "yes"), + NumericAttribute.defaultAttr, + NumericAttribute.defaultAttr) + val group = new AttributeGroup("user", attrs) + assert(group.size === 8) + assert(group.name === "user") + assert(group(0) === NumericAttribute.defaultAttr.withIndex(0)) + assert(group(2) === BinaryAttribute.defaultAttr.withIndex(2)) + assert(group.indexOf("age") === 3) + assert(group.indexOf("size") === 4) + assert(group.indexOf("clicked") === 5) + assert(!group.hasAttr("abc")) + intercept[NoSuchElementException] { + group("abc") + } + assert(group === AttributeGroup.fromMetadata(group.toMetadata, group.name)) + assert(group === AttributeGroup.fromStructField(group.toStructField())) + } + + test("attribute group without attributes") { + val group0 = new AttributeGroup("user", 10) + assert(group0.name === "user") + assert(group0.numAttributes === Some(10)) + assert(group0.size === 10) + assert(group0.attributes.isEmpty) + assert(group0 === AttributeGroup.fromMetadata(group0.toMetadata, group0.name)) + assert(group0 === AttributeGroup.fromStructField(group0.toStructField())) + + val group1 = new AttributeGroup("item") + assert(group1.name === "item") + assert(group1.numAttributes.isEmpty) + assert(group1.attributes.isEmpty) + assert(group1.size === -1) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala new file mode 100644 index 0000000000000..6ec35b03656f9 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala @@ -0,0 +1,212 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.attribute + +import org.scalatest.FunSuite + +import org.apache.spark.sql.types.{DoubleType, MetadataBuilder, Metadata} + +class AttributeSuite extends FunSuite { + + test("default numeric attribute") { + val attr: NumericAttribute = NumericAttribute.defaultAttr + val metadata = Metadata.fromJson("{}") + val metadataWithType = Metadata.fromJson("""{"type":"numeric"}""") + assert(attr.attrType === AttributeType.Numeric) + assert(attr.isNumeric) + assert(!attr.isNominal) + assert(attr.name.isEmpty) + assert(attr.index.isEmpty) + assert(attr.min.isEmpty) + assert(attr.max.isEmpty) + assert(attr.std.isEmpty) + assert(attr.sparsity.isEmpty) + assert(attr.toMetadata() === metadata) + assert(attr.toMetadata(withType = false) === metadata) + assert(attr.toMetadata(withType = true) === metadataWithType) + assert(attr === Attribute.fromMetadata(metadata)) + assert(attr === Attribute.fromMetadata(metadataWithType)) + intercept[NoSuchElementException] { + attr.toStructField() + } + } + + test("customized numeric attribute") { + val name = "age" + val index = 0 + val metadata = Metadata.fromJson("""{"name":"age","idx":0}""") + val metadataWithType = Metadata.fromJson("""{"type":"numeric","name":"age","idx":0}""") + val attr: NumericAttribute = NumericAttribute.defaultAttr + .withName(name) + .withIndex(index) + assert(attr.attrType == AttributeType.Numeric) + assert(attr.isNumeric) + assert(!attr.isNominal) + assert(attr.name === Some(name)) + assert(attr.index === Some(index)) + assert(attr.toMetadata() === metadata) + assert(attr.toMetadata(withType = false) === metadata) + assert(attr.toMetadata(withType = true) === metadataWithType) + assert(attr === Attribute.fromMetadata(metadata)) + assert(attr === Attribute.fromMetadata(metadataWithType)) + val field = attr.toStructField() + assert(field.dataType === DoubleType) + assert(!field.nullable) + assert(attr.withoutIndex === Attribute.fromStructField(field)) + val existingMetadata = new MetadataBuilder() + .putString("name", "test") + .build() + assert(attr.toStructField(existingMetadata).metadata.getString("name") === "test") + + val attr2 = + attr.withoutName.withoutIndex.withMin(0.0).withMax(1.0).withStd(0.5).withSparsity(0.3) + assert(attr2.name.isEmpty) + assert(attr2.index.isEmpty) + assert(attr2.min === Some(0.0)) + assert(attr2.max === Some(1.0)) + assert(attr2.std === Some(0.5)) + assert(attr2.sparsity === Some(0.3)) + assert(attr2 === Attribute.fromMetadata(attr2.toMetadata())) + } + + test("bad numeric attributes") { + val attr = NumericAttribute.defaultAttr + intercept[IllegalArgumentException](attr.withName("")) + intercept[IllegalArgumentException](attr.withIndex(-1)) + intercept[IllegalArgumentException](attr.withStd(-0.1)) + intercept[IllegalArgumentException](attr.withSparsity(-0.5)) + intercept[IllegalArgumentException](attr.withSparsity(1.5)) + } + + test("default nominal attribute") { + val attr: NominalAttribute = NominalAttribute.defaultAttr + val metadata = Metadata.fromJson("""{"type":"nominal"}""") + val metadataWithoutType = Metadata.fromJson("{}") + assert(attr.attrType === AttributeType.Nominal) + assert(!attr.isNumeric) + assert(attr.isNominal) + assert(attr.name.isEmpty) + assert(attr.index.isEmpty) + assert(attr.values.isEmpty) + assert(attr.numValues.isEmpty) + assert(attr.isOrdinal.isEmpty) + assert(attr.toMetadata() === metadata) + assert(attr.toMetadata(withType = true) === metadata) + assert(attr.toMetadata(withType = false) === metadataWithoutType) + assert(attr === Attribute.fromMetadata(metadata)) + assert(attr === NominalAttribute.fromMetadata(metadataWithoutType)) + intercept[NoSuchElementException] { + attr.toStructField() + } + } + + test("customized nominal attribute") { + val name = "size" + val index = 1 + val values = Array("small", "medium", "large") + val metadata = Metadata.fromJson( + """{"type":"nominal","name":"size","idx":1,"vals":["small","medium","large"]}""") + val metadataWithoutType = Metadata.fromJson( + """{"name":"size","idx":1,"vals":["small","medium","large"]}""") + val attr: NominalAttribute = NominalAttribute.defaultAttr + .withName(name) + .withIndex(index) + .withValues(values) + assert(attr.attrType === AttributeType.Nominal) + assert(!attr.isNumeric) + assert(attr.isNominal) + assert(attr.name === Some(name)) + assert(attr.index === Some(index)) + assert(attr.values === Some(values)) + assert(attr.indexOf("medium") === 1) + assert(attr.getValue(1) === "medium") + assert(attr.toMetadata() === metadata) + assert(attr.toMetadata(withType = true) === metadata) + assert(attr.toMetadata(withType = false) === metadataWithoutType) + assert(attr === Attribute.fromMetadata(metadata)) + assert(attr === NominalAttribute.fromMetadata(metadataWithoutType)) + assert(attr.withoutIndex === Attribute.fromStructField(attr.toStructField())) + + val attr2 = attr.withoutName.withoutIndex.withValues(attr.values.get :+ "x-large") + assert(attr2.name.isEmpty) + assert(attr2.index.isEmpty) + assert(attr2.values.get === Array("small", "medium", "large", "x-large")) + assert(attr2.indexOf("x-large") === 3) + assert(attr2 === Attribute.fromMetadata(attr2.toMetadata())) + assert(attr2 === NominalAttribute.fromMetadata(attr2.toMetadata(withType = false))) + } + + test("bad nominal attributes") { + val attr = NominalAttribute.defaultAttr + intercept[IllegalArgumentException](attr.withName("")) + intercept[IllegalArgumentException](attr.withIndex(-1)) + intercept[IllegalArgumentException](attr.withNumValues(-1)) + } + + test("default binary attribute") { + val attr = BinaryAttribute.defaultAttr + val metadata = Metadata.fromJson("""{"type":"binary"}""") + val metadataWithoutType = Metadata.fromJson("{}") + assert(attr.attrType === AttributeType.Binary) + assert(attr.isNumeric) + assert(attr.isNominal) + assert(attr.name.isEmpty) + assert(attr.index.isEmpty) + assert(attr.values.isEmpty) + assert(attr.toMetadata() === metadata) + assert(attr.toMetadata(withType = true) === metadata) + assert(attr.toMetadata(withType = false) === metadataWithoutType) + assert(attr === Attribute.fromMetadata(metadata)) + assert(attr === BinaryAttribute.fromMetadata(metadataWithoutType)) + intercept[NoSuchElementException] { + attr.toStructField() + } + } + + test("customized binary attribute") { + val name = "clicked" + val index = 2 + val values = Array("no", "yes") + val metadata = Metadata.fromJson( + """{"type":"binary","name":"clicked","idx":2,"vals":["no","yes"]}""") + val metadataWithoutType = Metadata.fromJson( + """{"name":"clicked","idx":2,"vals":["no","yes"]}""") + val attr = BinaryAttribute.defaultAttr + .withName(name) + .withIndex(index) + .withValues(values(0), values(1)) + assert(attr.attrType === AttributeType.Binary) + assert(attr.isNumeric) + assert(attr.isNominal) + assert(attr.name === Some(name)) + assert(attr.index === Some(index)) + assert(attr.values.get === values) + assert(attr.toMetadata() === metadata) + assert(attr.toMetadata(withType = true) === metadata) + assert(attr.toMetadata(withType = false) === metadataWithoutType) + assert(attr === Attribute.fromMetadata(metadata)) + assert(attr === BinaryAttribute.fromMetadata(metadataWithoutType)) + assert(attr.withoutIndex === Attribute.fromStructField(attr.toStructField())) + } + + test("bad binary attributes") { + val attr = BinaryAttribute.defaultAttr + intercept[IllegalArgumentException](attr.withName("")) + intercept[IllegalArgumentException](attr.withIndex(-1)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index 64dcc0fb9f82c..5a27c7d2309c5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -85,6 +85,14 @@ class NaiveBayesSuite extends FunSuite with MLlibTestSparkContext { assert(numOfPredictions < input.length / 5) } + test("get, set params") { + val nb = new NaiveBayes() + nb.setLambda(2.0) + assert(nb.getLambda === 2.0) + nb.setLambda(3.0) + assert(nb.getLambda === 3.0) + } + test("Naive Bayes") { val nPoints = 10000 diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala index 82c327bd49fcd..22855e4e8f247 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala @@ -55,7 +55,7 @@ class NNLSSuite extends FunSuite { for (k <- 0 until 100) { val (ata, atb) = genOnesData(n, rand) - val x = new DoubleMatrix(NNLS.solve(ata, atb, ws)) + val x = new DoubleMatrix(NNLS.solve(ata.data, atb.data, ws)) assert(x.length === n) val answer = DoubleMatrix.ones(n, 1) SimpleBlas.axpy(-1.0, answer, x) @@ -79,7 +79,7 @@ class NNLSSuite extends FunSuite { val goodx = Array(0.13025, 0.54506, 0.2874, 0.0, 0.028628) val ws = NNLS.createWorkspace(n) - val x = NNLS.solve(ata, atb, ws) + val x = NNLS.solve(ata.data, atb.data, ws) for (i <- 0 until n) { assert(x(i) ~== goodx(i) absTol 1E-3) assert(x(i) >= 0) @@ -104,7 +104,7 @@ class NNLSSuite extends FunSuite { val ws = NNLS.createWorkspace(n) - val x = new DoubleMatrix(NNLS.solve(ata, atb, ws)) + val x = new DoubleMatrix(NNLS.solve(ata.data, atb.data, ws)) val obj = computeObjectiveValue(ata, atb, x) assert(obj < refObj + 1E-5) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala index f6a1e19f50296..16ecae23dd9d4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala @@ -21,9 +21,9 @@ import org.scalatest.FunSuite import org.apache.commons.math3.distribution.NormalDistribution -import org.apache.spark.mllib.util.LocalClusterSparkContext +import org.apache.spark.mllib.util.MLlibTestSparkContext -class KernelDensitySuite extends FunSuite with LocalClusterSparkContext { +class KernelDensitySuite extends FunSuite with MLlibTestSparkContext { test("kernel density single sample") { val rdd = sc.parallelize(Array(5.0)) val evaluationPoints = Array(5.0, 6.0) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java b/network/common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java rename to network/common/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslMessage.java b/network/common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslMessage.java rename to network/common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java b/network/common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java rename to network/common/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java b/network/common/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java rename to network/common/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java b/network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java rename to network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java b/network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java similarity index 100% rename from network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java rename to network/common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java diff --git a/network/shuffle/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java b/network/common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java similarity index 100% rename from network/shuffle/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java rename to network/common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java diff --git a/pom.xml b/pom.xml index a19da73cf45b3..6fc56a86d44ac 100644 --- a/pom.xml +++ b/pom.xml @@ -157,6 +157,7 @@ 1.8.8 2.4.4 1.1.1.6 + 1.1.2