diff --git a/.rat-excludes b/.rat-excludes index 17cf6d0ed1cf3..85bfad60fcadc 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -39,3 +39,4 @@ work .*\.q golden test.out/* +.*iml diff --git a/assembly/pom.xml b/assembly/pom.xml index b5e752c6cd1f6..923bf47f7076a 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -163,6 +163,16 @@ + + hive + + + org.apache.spark + spark-hive_${scala.binary.version} + ${project.version} + + + spark-ganglia-lgpl @@ -208,7 +218,7 @@ org.codehaus.mojo buildnumber-maven-plugin - 1.1 + 1.2 validate diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index bef42df71ce01..be37102dc069a 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -30,21 +30,7 @@ FWDIR="$(cd `dirname $0`/..; pwd)" # Build up classpath CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" -# Support for interacting with Hive. Since hive pulls in a lot of dependencies that might break -# existing Spark applications, it is not included in the standard spark assembly. Instead, we only -# include it in the classpath if the user has explicitly requested it by running "sbt hive/assembly" -# Hopefully we will find a way to avoid uber-jars entirely and deploy only the needed packages in -# the future. -if [ -f "$FWDIR"/sql/hive/target/scala-$SCALA_VERSION/spark-hive-assembly-*.jar ]; then - - # Datanucleus jars do not work if only included in the uberjar as plugin.xml metadata is lost. - DATANUCLEUSJARS=$(JARS=("$FWDIR/lib_managed/jars"/datanucleus-*.jar); IFS=:; echo "${JARS[*]}") - CLASSPATH=$CLASSPATH:$DATANUCLEUSJARS - - ASSEMBLY_DIR="$FWDIR/sql/hive/target/scala-$SCALA_VERSION/" -else - ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION/" -fi +ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION" # First check if we have a dependencies jar. If so, include binary classes with the deps jar if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then @@ -59,7 +45,7 @@ if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" - DEPS_ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark*-assembly*hadoop*-deps.jar` + DEPS_ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar` CLASSPATH="$CLASSPATH:$DEPS_ASSEMBLY_JAR" else # Else use spark-assembly jar from either RELEASE or assembly directory @@ -71,6 +57,23 @@ else CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" fi +# When Hive support is needed, Datanucleus jars must be included on the classpath. +# Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. +# Both sbt and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is +# built with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark +# assembly is built for Hive, before actually populating the CLASSPATH with the jars. +# Note that this check order is faster (by up to half a second) in the case where Hive is not used. +num_datanucleus_jars=$(ls "$FWDIR"/lib_managed/jars/ | grep "datanucleus-.*\\.jar" | wc -l) +if [ $num_datanucleus_jars -gt 0 ]; then + AN_ASSEMBLY_JAR=${ASSEMBLY_JAR:-$DEPS_ASSEMBLY_JAR} + num_hive_files=$(jar tvf "$AN_ASSEMBLY_JAR" org/apache/hadoop/hive/ql/exec 2>/dev/null | wc -l) + if [ $num_hive_files -gt 0 ]; then + echo "Spark assembly has been built with Hive, including Datanucleus jars on classpath" 1>&2 + DATANUCLEUSJARS=$(echo "$FWDIR/lib_managed/jars"/datanucleus-*.jar | tr " " :) + CLASSPATH=$CLASSPATH:$DATANUCLEUSJARS + fi +fi + # Add test classes if we're running from SBT or Maven with SPARK_TESTING set to 1 if [[ $SPARK_TESTING == 1 ]]; then CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/test-classes" diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index 476dd826551fd..d425f9feaac54 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -30,6 +30,9 @@ if [ -z "$SPARK_ENV_LOADED" ]; then use_conf_dir=${SPARK_CONF_DIR:-"$parent_dir/conf"} if [ -f "${use_conf_dir}/spark-env.sh" ]; then + # Promote all variable declarations to environment (exported) variables + set -a . "${use_conf_dir}/spark-env.sh" + set +a fi fi diff --git a/bin/spark-class b/bin/spark-class index 0dcf0e156cb52..76fde3e448891 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -154,5 +154,3 @@ if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then fi exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" - - diff --git a/bin/spark-shell b/bin/spark-shell index fac006cf492ed..ea12d256b23a1 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -34,7 +34,7 @@ set -o posix FWDIR="$(cd `dirname $0`/..; pwd)" SPARK_REPL_OPTS="${SPARK_REPL_OPTS:-""}" -DEFAULT_MASTER="local" +DEFAULT_MASTER="local[*]" MASTER=${MASTER:-""} info_log=0 @@ -64,7 +64,7 @@ ${txtbld}OPTIONS${txtrst}: is followed by m for megabytes or g for gigabytes, e.g. "1g". -dm --driver-memory : The memory used by the Spark Shell, the number is followed by m for megabytes or g for gigabytes, e.g. "1g". - -m --master : A full string that describes the Spark Master, defaults to "local" + -m --master : A full string that describes the Spark Master, defaults to "local[*]" e.g. "spark://localhost:7077". --log-conf : Enables logging of the supplied SparkConf as INFO at start of the Spark Context. @@ -127,7 +127,7 @@ function set_spark_log_conf(){ function set_spark_master(){ if ! [[ "$1" =~ $ARG_FLAG_PATTERN ]]; then - MASTER="$1" + export MASTER="$1" else out_error "wrong format for $2" fi @@ -145,7 +145,7 @@ function resolve_spark_master(){ fi if [ -z "$MASTER" ]; then - MASTER="$DEFAULT_MASTER" + export MASTER="$DEFAULT_MASTER" fi } diff --git a/core/pom.xml b/core/pom.xml index e4c32eff0cd77..1f808380817c9 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -117,12 +117,10 @@ com.twitter chill_${scala.binary.version} - 0.3.1 com.twitter chill-java - 0.3.1 commons-net @@ -200,6 +198,53 @@ derby test + + org.tachyonproject + tachyon + 0.4.1-thrift + + + org.apache.hadoop + hadoop-client + + + org.apache.curator + curator-recipes + + + org.eclipse.jetty + jetty-jsp + + + org.eclipse.jetty + jetty-webapp + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-servlet + + + junit + junit + + + org.powermock + powermock-module-junit4 + + + org.powermock + powermock-api-mockito + + + org.apache.curator + curator-test + + + org.scalatest scalatest_${scala.binary.version} diff --git a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java index 9f13b39909481..840a1bd93bfbb 100644 --- a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java +++ b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java @@ -23,17 +23,18 @@ * Expose some commonly useful storage level constants. */ public class StorageLevels { - public static final StorageLevel NONE = create(false, false, false, 1); - public static final StorageLevel DISK_ONLY = create(true, false, false, 1); - public static final StorageLevel DISK_ONLY_2 = create(true, false, false, 2); - public static final StorageLevel MEMORY_ONLY = create(false, true, true, 1); - public static final StorageLevel MEMORY_ONLY_2 = create(false, true, true, 2); - public static final StorageLevel MEMORY_ONLY_SER = create(false, true, false, 1); - public static final StorageLevel MEMORY_ONLY_SER_2 = create(false, true, false, 2); - public static final StorageLevel MEMORY_AND_DISK = create(true, true, true, 1); - public static final StorageLevel MEMORY_AND_DISK_2 = create(true, true, true, 2); - public static final StorageLevel MEMORY_AND_DISK_SER = create(true, true, false, 1); - public static final StorageLevel MEMORY_AND_DISK_SER_2 = create(true, true, false, 2); + public static final StorageLevel NONE = create(false, false, false, false, 1); + public static final StorageLevel DISK_ONLY = create(true, false, false, false, 1); + public static final StorageLevel DISK_ONLY_2 = create(true, false, false, false, 2); + public static final StorageLevel MEMORY_ONLY = create(false, true, false, true, 1); + public static final StorageLevel MEMORY_ONLY_2 = create(false, true, false, true, 2); + public static final StorageLevel MEMORY_ONLY_SER = create(false, true, false, false, 1); + public static final StorageLevel MEMORY_ONLY_SER_2 = create(false, true, false, false, 2); + public static final StorageLevel MEMORY_AND_DISK = create(true, true, false, true, 1); + public static final StorageLevel MEMORY_AND_DISK_2 = create(true, true, false, true, 2); + public static final StorageLevel MEMORY_AND_DISK_SER = create(true, true, false, false, 1); + public static final StorageLevel MEMORY_AND_DISK_SER_2 = create(true, true, false, false, 2); + public static final StorageLevel OFF_HEAP = create(false, false, true, false, 1); /** * Create a new StorageLevel object. @@ -42,7 +43,26 @@ public class StorageLevels { * @param deserialized saved as deserialized objects, if true * @param replication replication factor */ - public static StorageLevel create(boolean useDisk, boolean useMemory, boolean deserialized, int replication) { - return StorageLevel.apply(useDisk, useMemory, deserialized, replication); + @Deprecated + public static StorageLevel create(boolean useDisk, boolean useMemory, boolean deserialized, + int replication) { + return StorageLevel.apply(useDisk, useMemory, false, deserialized, replication); + } + + /** + * Create a new StorageLevel object. + * @param useDisk saved to disk, if true + * @param useMemory saved to memory, if true + * @param useOffHeap saved to Tachyon, if true + * @param deserialized saved as deserialized objects, if true + * @param replication replication factor + */ + public static StorageLevel create( + boolean useDisk, + boolean useMemory, + boolean useOffHeap, + boolean deserialized, + int replication) { + return StorageLevel.apply(useDisk, useMemory, useOffHeap, deserialized, replication); } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c8d659d656ef4..dfc173357c12b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -19,14 +19,13 @@ package org.apache.spark import java.io._ import java.net.URI -import java.util.{Properties, UUID} import java.util.concurrent.atomic.AtomicInteger - +import java.util.{Properties, UUID} +import java.util.UUID.randomUUID import scala.collection.{Map, Set} import scala.collection.generic.Growable import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.reflect.{ClassTag, classTag} - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} @@ -37,6 +36,7 @@ import org.apache.mesos.MesosNativeLibrary import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} +import org.apache.spark.input.WholeTextFileInputFormat import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.scheduler._ @@ -129,6 +129,11 @@ class SparkContext( val master = conf.get("spark.master") val appName = conf.get("spark.app.name") + // Generate the random name for a temp folder in Tachyon + // Add a timestamp as the suffix here to make it more safe + val tachyonFolderName = "spark-" + randomUUID.toString() + conf.set("spark.tachyonStore.folderName", tachyonFolderName) + val isLocal = (master == "local" || master.startsWith("local[")) if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") @@ -378,6 +383,39 @@ class SparkContext( minSplits).map(pair => pair._2.toString) } + /** + * Read a directory of text files from HDFS, a local file system (available on all nodes), or any + * Hadoop-supported file system URI. Each file is read as a single record and returned in a + * key-value pair, where the key is the path of each file, the value is the content of each file. + * + *

For example, if you have the following files: + * {{{ + * hdfs://a-hdfs-path/part-00000 + * hdfs://a-hdfs-path/part-00001 + * ... + * hdfs://a-hdfs-path/part-nnnnn + * }}} + * + * Do `val rdd = sparkContext.wholeTextFile("hdfs://a-hdfs-path")`, + * + *

then `rdd` contains + * {{{ + * (a-hdfs-path/part-00000, its content) + * (a-hdfs-path/part-00001, its content) + * ... + * (a-hdfs-path/part-nnnnn, its content) + * }}} + * + * @note Small files are preferred, as each file will be loaded fully in memory. + */ + def wholeTextFiles(path: String): RDD[(String, String)] = { + newAPIHadoopFile( + path, + classOf[WholeTextFileInputFormat], + classOf[String], + classOf[String]) + } + /** * Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf given its InputFormat and other * necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable), @@ -704,10 +742,6 @@ class SparkContext( */ def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap - def getStageInfo: Map[Stage, StageInfo] = { - dagScheduler.stageToInfos - } - /** * Return information about blocks stored in all of the slaves */ @@ -1262,8 +1296,8 @@ object SparkContext extends Logging { /** Creates a task scheduler based on a given master URL. Extracted for testing. */ private def createTaskScheduler(sc: SparkContext, master: String): TaskScheduler = { - // Regular expression used for local[N] master format - val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r + // Regular expression used for local[N] and local[*] master formats + val LOCAL_N_REGEX = """local\[([0-9\*]+)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+)\s*,\s*([0-9]+)\]""".r // Regular expression for simulating a Spark cluster of [N, cores, memory] locally @@ -1286,8 +1320,11 @@ object SparkContext extends Logging { scheduler case LOCAL_N_REGEX(threads) => + def localCpuCount = Runtime.getRuntime.availableProcessors() + // local[*] estimates the number of cores on the machine; local[N] uses exactly N threads. + val threadCount = if (threads == "*") localCpuCount else threads.toInt val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalBackend(scheduler, threads.toInt) + val backend = new LocalBackend(scheduler, threadCount) scheduler.initialize(backend) scheduler 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 e03b8e78d5f52..6e8ec8e0c7629 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 @@ -17,7 +17,8 @@ package org.apache.spark.api.java -import java.util.{Comparator, List => JList} +import java.util.{Comparator, Iterator => JIterator, List => JList} +import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -280,6 +281,17 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { new java.util.ArrayList(arr) } + /** + * Return an iterator that contains all of the elements in this RDD. + * + * The iterator will consume as much memory as the largest partition in this RDD. + */ + def toLocalIterator(): JIterator[T] = { + import scala.collection.JavaConversions._ + rdd.toLocalIterator + } + + /** * Return an array that contains all of the elements in this RDD. * @deprecated As of Spark 1.0.0, toArray() is deprecated, use {@link #collect()} instead diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index e531a57aced31..a2855d4db1d2e 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -154,6 +154,34 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork */ def textFile(path: String, minSplits: Int): JavaRDD[String] = sc.textFile(path, minSplits) + /** + * Read a directory of text files from HDFS, a local file system (available on all nodes), or any + * Hadoop-supported file system URI. Each file is read as a single record and returned in a + * key-value pair, where the key is the path of each file, the value is the content of each file. + * + *

For example, if you have the following files: + * {{{ + * hdfs://a-hdfs-path/part-00000 + * hdfs://a-hdfs-path/part-00001 + * ... + * hdfs://a-hdfs-path/part-nnnnn + * }}} + * + * Do `JavaPairRDD rdd = sparkContext.wholeTextFiles("hdfs://a-hdfs-path")`, + * + *

then `rdd` contains + * {{{ + * (a-hdfs-path/part-00000, its content) + * (a-hdfs-path/part-00001, its content) + * ... + * (a-hdfs-path/part-nnnnn, its content) + * }}} + * + * @note Small files are preferred, as each file will be loaded fully in memory. + */ + def wholeTextFiles(path: String): JavaPairRDD[String, String] = + new JavaPairRDD(sc.wholeTextFiles(path)) + /** Get an RDD for a Hadoop SequenceFile with given key and value types. * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each 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 b67286a4e3b75..32f1100406d74 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 @@ -19,6 +19,7 @@ package org.apache.spark.api.python import java.io._ import java.net._ +import java.nio.charset.Charset import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} import scala.collection.JavaConversions._ @@ -206,6 +207,7 @@ private object SpecialLengths { } private[spark] object PythonRDD { + val UTF8 = Charset.forName("UTF-8") def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int): JavaRDD[Array[Byte]] = { @@ -266,7 +268,7 @@ private[spark] object PythonRDD { } def writeUTF(str: String, dataOut: DataOutputStream) { - val bytes = str.getBytes("UTF-8") + val bytes = str.getBytes(UTF8) dataOut.writeInt(bytes.length) dataOut.write(bytes) } @@ -286,7 +288,7 @@ private[spark] object PythonRDD { private class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] { - override def call(arr: Array[Byte]) : String = new String(arr, "UTF-8") + override def call(arr: Array[Byte]) : String = new String(arr, PythonRDD.UTF8) } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 83ce14a0a806a..a7368f9f3dfbe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -86,6 +86,10 @@ private[deploy] object DeployMessages { case class KillDriver(driverId: String) extends DeployMessage + // Worker internal + + case object WorkDirCleanup // Sent to Worker actor periodically for cleaning up app folders + // AppClient to Master case class RegisterApplication(appDescription: ApplicationDescription) 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 1fa799190409f..e05fbfe321495 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -79,20 +79,23 @@ object SparkSubmit { printErrorAndExit("master must start with yarn, mesos, spark, or local") } - // Because "yarn-standalone" and "yarn-client" encapsulate both the master + // Because "yarn-cluster" and "yarn-client" encapsulate both the master // and deploy mode, we have some logic to infer the master and deploy mode // from each other if only one is specified, or exit early if they are at odds. - if (appArgs.deployMode == null && appArgs.master == "yarn-standalone") { + if (appArgs.deployMode == null && + (appArgs.master == "yarn-standalone" || appArgs.master == "yarn-cluster")) { appArgs.deployMode = "cluster" } if (appArgs.deployMode == "cluster" && appArgs.master == "yarn-client") { printErrorAndExit("Deploy mode \"cluster\" and master \"yarn-client\" are not compatible") } - if (appArgs.deployMode == "client" && appArgs.master == "yarn-standalone") { - printErrorAndExit("Deploy mode \"client\" and master \"yarn-standalone\" are not compatible") + if (appArgs.deployMode == "client" && + (appArgs.master == "yarn-standalone" || appArgs.master == "yarn-cluster")) { + printErrorAndExit("Deploy mode \"client\" and master \"" + appArgs.master + + "\" are not compatible") } if (appArgs.deployMode == "cluster" && appArgs.master.startsWith("yarn")) { - appArgs.master = "yarn-standalone" + appArgs.master = "yarn-cluster" } if (appArgs.deployMode != "cluster" && appArgs.master.startsWith("yarn")) { appArgs.master = "yarn-client" 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 9c8f54ea6f77a..834b3df2f164b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -171,7 +171,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { outStream.println("Unknown/unsupported param " + unknownParam) } outStream.println( - """Usage: spark-submit [options] + """Usage: spark-submit [options] |Options: | --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. | --deploy-mode DEPLOY_MODE Mode to deploy the app in, either 'client' or 'cluster'. 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 8a71ddda4cb5e..bf5a8d09dd2df 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 @@ -64,6 +64,12 @@ private[spark] class Worker( val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 + val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", true) + // How often worker will clean up old app folders + 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) + // Index into masterUrls that we're currently trying to register with. var masterIndex = 0 @@ -179,12 +185,28 @@ private[spark] class Worker( registered = true changeMaster(masterUrl, masterWebUiUrl) context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis, self, SendHeartbeat) + if (CLEANUP_ENABLED) { + context.system.scheduler.schedule(CLEANUP_INTERVAL_MILLIS millis, + CLEANUP_INTERVAL_MILLIS millis, self, WorkDirCleanup) + } case SendHeartbeat => masterLock.synchronized { if (connected) { master ! Heartbeat(workerId) } } + case WorkDirCleanup => + // Spin up a separate thread (in a future) to do the dir cleanup; don't tie up worker actor + val cleanupFuture = concurrent.future { + logInfo("Cleaning up oldest application directories in " + workDir + " ...") + Utils.findOldFiles(workDir, APP_DATA_RETENTION_SECS) + .foreach(Utils.deleteRecursively) + } + cleanupFuture onFailure { + case e: Throwable => + logError("App dir cleanup failed: " + e.getMessage, e) + } + case MasterChanged(masterUrl, masterWebUiUrl) => logInfo("Master has changed, new master is at " + masterUrl) changeMaster(masterUrl, masterWebUiUrl) @@ -331,7 +353,6 @@ private[spark] class Worker( } private[spark] object Worker { - def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 3486092a140fb..16887d8892b31 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -53,7 +53,8 @@ private[spark] class CoarseGrainedExecutorBackend( case RegisteredExecutor(sparkProperties) => logInfo("Successfully registered with driver") // Make this host instead of hostPort ? - executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties) + executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties, + false) case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) @@ -105,7 +106,8 @@ private[spark] object CoarseGrainedExecutorBackend { // set it val sparkHostPort = hostname + ":" + boundPort actorSystem.actorOf( - Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, sparkHostPort, cores), + Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, + sparkHostPort, cores), name = "Executor") workerUrl.foreach{ url => actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala index 210f3dbeebaca..ceff3a067d72a 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala @@ -41,6 +41,12 @@ object ExecutorExitCode { /** DiskStore failed to create a local temporary directory after many attempts. */ val DISK_STORE_FAILED_TO_CREATE_DIR = 53 + /** TachyonStore failed to initialize after many attempts. */ + val TACHYON_STORE_FAILED_TO_INITIALIZE = 54 + + /** TachyonStore failed to create a local temporary directory after many attempts. */ + val TACHYON_STORE_FAILED_TO_CREATE_DIR = 55 + def explainExitCode(exitCode: Int): String = { exitCode match { case UNCAUGHT_EXCEPTION => "Uncaught exception" @@ -48,6 +54,9 @@ object ExecutorExitCode { case OOM => "OutOfMemoryError" case DISK_STORE_FAILED_TO_CREATE_DIR => "Failed to create local directory (bad spark.local.dir?)" + case TACHYON_STORE_FAILED_TO_INITIALIZE => "TachyonStore failed to initialize." + case TACHYON_STORE_FAILED_TO_CREATE_DIR => + "TachyonStore failed to create a local temporary directory." case _ => "Unknown executor exit code (" + exitCode + ")" + ( if (exitCode > 128) { diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala new file mode 100644 index 0000000000000..4887fb6b84eb2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala @@ -0,0 +1,47 @@ +/* + * 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.input + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.InputSplit +import org.apache.hadoop.mapreduce.JobContext +import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat +import org.apache.hadoop.mapreduce.RecordReader +import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader +import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit + +/** + * A [[org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat CombineFileInputFormat]] for + * reading whole text files. Each file is read as key-value pair, where the key is the file path and + * the value is the entire content of file. + */ + +private[spark] class WholeTextFileInputFormat extends CombineFileInputFormat[String, String] { + override protected def isSplitable(context: JobContext, file: Path): Boolean = false + + override def createRecordReader( + split: InputSplit, + context: TaskAttemptContext): RecordReader[String, String] = { + + new CombineFileRecordReader[String, String]( + split.asInstanceOf[CombineFileSplit], + context, + classOf[WholeTextFileRecordReader]) + } +} diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala new file mode 100644 index 0000000000000..c3dabd2e79995 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala @@ -0,0 +1,72 @@ +/* + * 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.input + +import com.google.common.io.{ByteStreams, Closeables} + +import org.apache.hadoop.io.Text +import org.apache.hadoop.mapreduce.InputSplit +import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit +import org.apache.hadoop.mapreduce.RecordReader +import org.apache.hadoop.mapreduce.TaskAttemptContext + +/** + * A [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] for reading a single whole text file + * out in a key-value pair, where the key is the file path and the value is the entire content of + * the file. + */ +private[spark] class WholeTextFileRecordReader( + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) + extends RecordReader[String, String] { + + private val path = split.getPath(index) + private val fs = path.getFileSystem(context.getConfiguration) + + // True means the current file has been processed, then skip it. + private var processed = false + + private val key = path.toString + private var value: String = null + + override def initialize(split: InputSplit, context: TaskAttemptContext) = {} + + override def close() = {} + + override def getProgress = if (processed) 1.0f else 0.0f + + override def getCurrentKey = key + + override def getCurrentValue = value + + override def nextKeyValue = { + if (!processed) { + val fileIn = fs.open(path) + val innerBuffer = ByteStreams.toByteArray(fileIn) + + value = new Text(innerBuffer).toString + Closeables.close(fileIn, false) + + processed = true + true + } else { + false + } + } +} diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 4250a9d02f764..41ae0fec823e7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -17,6 +17,9 @@ package org.apache.spark.rdd +import java.io.File +import java.io.FilenameFilter +import java.io.IOException import java.io.PrintWriter import java.util.StringTokenizer @@ -27,6 +30,7 @@ import scala.io.Source import scala.reflect.ClassTag import org.apache.spark.{Partition, SparkEnv, TaskContext} +import org.apache.spark.util.Utils /** @@ -38,7 +42,8 @@ class PipedRDD[T: ClassTag]( command: Seq[String], envVars: Map[String, String], printPipeContext: (String => Unit) => Unit, - printRDDElement: (T, String => Unit) => Unit) + printRDDElement: (T, String => Unit) => Unit, + separateWorkingDir: Boolean) extends RDD[String](prev) { // Similar to Runtime.exec(), if we are given a single string, split it into words @@ -48,12 +53,24 @@ class PipedRDD[T: ClassTag]( command: String, envVars: Map[String, String] = Map(), printPipeContext: (String => Unit) => Unit = null, - printRDDElement: (T, String => Unit) => Unit = null) = - this(prev, PipedRDD.tokenize(command), envVars, printPipeContext, printRDDElement) + printRDDElement: (T, String => Unit) => Unit = null, + separateWorkingDir: Boolean = false) = + this(prev, PipedRDD.tokenize(command), envVars, printPipeContext, printRDDElement, + separateWorkingDir) override def getPartitions: Array[Partition] = firstParent[T].partitions + /** + * A FilenameFilter that accepts anything that isn't equal to the name passed in. + * @param name of file or directory to leave out + */ + class NotEqualsFileNameFilter(filterName: String) extends FilenameFilter { + def accept(dir: File, name: String): Boolean = { + !name.equals(filterName) + } + } + override def compute(split: Partition, context: TaskContext): Iterator[String] = { val pb = new ProcessBuilder(command) // Add the environmental variables to the process. @@ -67,6 +84,38 @@ class PipedRDD[T: ClassTag]( currentEnvVars.putAll(hadoopSplit.getPipeEnvVars()) } + // When spark.worker.separated.working.directory option is turned on, each + // task will be run in separate directory. This should be resolve file + // access conflict issue + val taskDirectory = "./tasks/" + java.util.UUID.randomUUID.toString + var workInTaskDirectory = false + logDebug("taskDirectory = " + taskDirectory) + if (separateWorkingDir == true) { + val currentDir = new File(".") + logDebug("currentDir = " + currentDir.getAbsolutePath()) + val taskDirFile = new File(taskDirectory) + taskDirFile.mkdirs() + + try { + val tasksDirFilter = new NotEqualsFileNameFilter("tasks") + + // Need to add symlinks to jars, files, and directories. On Yarn we could have + // directories and other files not known to the SparkContext that were added via the + // Hadoop distributed cache. We also don't want to symlink to the /tasks directories we + // are creating here. + for (file <- currentDir.list(tasksDirFilter)) { + val fileWithDir = new File(currentDir, file) + Utils.symlink(new File(fileWithDir.getAbsolutePath()), + new File(taskDirectory + "/" + fileWithDir.getName())) + } + pb.directory(taskDirFile) + workInTaskDirectory = true + } catch { + case e: Exception => logError("Unable to setup task working directory: " + e.getMessage + + " (" + taskDirectory + ")") + } + } + val proc = pb.start() val env = SparkEnv.get @@ -112,6 +161,15 @@ class PipedRDD[T: ClassTag]( if (exitStatus != 0) { throw new Exception("Subprocess exited with status " + exitStatus) } + + // cleanup task working directory if used + if (workInTaskDirectory == true) { + scala.util.control.Exception.ignoring(classOf[IOException]) { + Utils.deleteRecursively(new File(taskDirectory)) + } + logDebug("Removed task working directory " + taskDirectory) + } + false } } 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 50dbbe35f3745..bf3c57ad41eb2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -483,16 +483,19 @@ abstract class RDD[T: ClassTag]( * instead of constructing a huge String to concat all the elements: * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = * for (e <- record._2){f(e)} + * @param separateWorkingDir Use separate working directories for each task. * @return the result RDD */ def pipe( command: Seq[String], env: Map[String, String] = Map(), printPipeContext: (String => Unit) => Unit = null, - printRDDElement: (T, String => Unit) => Unit = null): RDD[String] = { + printRDDElement: (T, String => Unit) => Unit = null, + separateWorkingDir: Boolean = false): RDD[String] = { new PipedRDD(this, command, env, if (printPipeContext ne null) sc.clean(printPipeContext) else null, - if (printRDDElement ne null) sc.clean(printRDDElement) else null) + if (printRDDElement ne null) sc.clean(printRDDElement) else null, + separateWorkingDir) } /** @@ -660,6 +663,18 @@ abstract class RDD[T: ClassTag]( Array.concat(results: _*) } + /** + * Return an iterator that contains all of the elements in this RDD. + * + * The iterator will consume as much memory as the largest partition in this RDD. + */ + def toLocalIterator: Iterator[T] = { + def collectPartition(p: Int): Array[T] = { + sc.runJob(this, (iter: Iterator[T]) => iter.toArray, Seq(p), allowLocal = false).head + } + (0 until partitions.length).iterator.flatMap(i => collectPartition(i)) + } + /** * Return an array that contains all of the elements in this RDD. */ 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 0ccc1339aab41..6368665f249ee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -86,7 +86,7 @@ class DAGScheduler( private[scheduler] val shuffleToMapStage = new HashMap[Int, Stage] private[scheduler] val jobIdToActiveJob = new HashMap[Int, ActiveJob] private[scheduler] val resultStageToJob = new HashMap[Stage, ActiveJob] - private[spark] val stageToInfos = new HashMap[Stage, StageInfo] + private[scheduler] val stageToInfos = new HashMap[Stage, StageInfo] // Stages we need to run whose parents aren't done private[scheduler] val waitingStages = new HashSet[Stage] diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index 5555585c8b4cd..b3f2cb346f7da 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -164,8 +164,7 @@ object InputFormatInfo { PS: I know the wording here is weird, hopefully it makes some sense ! */ - def computePreferredLocations(formats: Seq[InputFormatInfo]): HashMap[String, HashSet[SplitInfo]] - = { + def computePreferredLocations(formats: Seq[InputFormatInfo]): Map[String, Set[SplitInfo]] = { val nodeToSplit = new HashMap[String, HashSet[SplitInfo]] for (inputSplit <- formats) { @@ -178,6 +177,6 @@ object InputFormatInfo { } } - nodeToSplit + nodeToSplit.mapValues(_.toSet).toMap } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala index ef924123a3b11..c8f397609a0b4 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala @@ -79,5 +79,3 @@ private object BlockInfo { private val BLOCK_PENDING: Long = -1L private val BLOCK_FAILED: Long = -2L } - -private[spark] case class BlockStatus(storageLevel: StorageLevel, memSize: Long, diskSize: Long) 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 e684831c00abc..a9e3e48767b1b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -29,7 +29,7 @@ import akka.actor.{ActorSystem, Cancellable, Props} import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} import sun.nio.ch.DirectBuffer -import org.apache.spark._ +import org.apache.spark.{Logging, MapOutputTracker, SecurityManager, SparkConf, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer @@ -60,6 +60,17 @@ private[spark] class BlockManager( private[storage] val memoryStore = new MemoryStore(this, maxMemory) private[storage] val diskStore = new DiskStore(this, diskBlockManager) + var tachyonInitialized = false + private[storage] lazy val tachyonStore: TachyonStore = { + val storeDir = conf.get("spark.tachyonStore.baseDir", "/tmp_spark_tachyon") + val appFolderName = conf.get("spark.tachyonStore.folderName") + val tachyonStorePath = s"${storeDir}/${appFolderName}/${this.executorId}" + val tachyonMaster = conf.get("spark.tachyonStore.url", "tachyon://localhost:19998") + val tachyonBlockManager = new TachyonBlockManager( + shuffleBlockManager, tachyonStorePath, tachyonMaster) + tachyonInitialized = true + new TachyonStore(this, tachyonBlockManager) + } // If we use Netty for shuffle, start a new Netty-based shuffle sender service. private val nettyPort: Int = { @@ -214,7 +225,8 @@ private[spark] class BlockManager( blockInfo.get(blockId).map { info => val memSize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L val diskSize = if (diskStore.contains(blockId)) diskStore.getSize(blockId) else 0L - BlockStatus(info.level, memSize, diskSize) + val tachyonSize = if (tachyonStore.contains(blockId)) tachyonStore.getSize(blockId) else 0L + BlockStatus(info.level, memSize, diskSize, tachyonSize) } } @@ -263,8 +275,10 @@ private[spark] class BlockManager( if (info.tellMaster) { val storageLevel = status.storageLevel val inMemSize = Math.max(status.memSize, droppedMemorySize) + val inTachyonSize = status.tachyonSize val onDiskSize = status.diskSize - master.updateBlockInfo(blockManagerId, blockId, storageLevel, inMemSize, onDiskSize) + master.updateBlockInfo( + blockManagerId, blockId, storageLevel, inMemSize, onDiskSize, inTachyonSize) } else true } @@ -274,22 +288,24 @@ private[spark] class BlockManager( * and the updated in-memory and on-disk sizes. */ private def getCurrentBlockStatus(blockId: BlockId, info: BlockInfo): BlockStatus = { - val (newLevel, inMemSize, onDiskSize) = info.synchronized { + val (newLevel, inMemSize, onDiskSize, inTachyonSize) = info.synchronized { info.level match { case null => - (StorageLevel.NONE, 0L, 0L) + (StorageLevel.NONE, 0L, 0L, 0L) case level => val inMem = level.useMemory && memoryStore.contains(blockId) + val inTachyon = level.useOffHeap && tachyonStore.contains(blockId) val onDisk = level.useDisk && diskStore.contains(blockId) val deserialized = if (inMem) level.deserialized else false - val replication = if (inMem || onDisk) level.replication else 1 - val storageLevel = StorageLevel(onDisk, inMem, deserialized, replication) + val replication = if (inMem || inTachyon || onDisk) level.replication else 1 + val storageLevel = StorageLevel(onDisk, inMem, inTachyon, deserialized, replication) val memSize = if (inMem) memoryStore.getSize(blockId) else 0L + val tachyonSize = if (inTachyon) tachyonStore.getSize(blockId) else 0L val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L - (storageLevel, memSize, diskSize) + (storageLevel, memSize, diskSize, tachyonSize) } } - BlockStatus(newLevel, inMemSize, onDiskSize) + BlockStatus(newLevel, inMemSize, onDiskSize, inTachyonSize) } /** @@ -369,6 +385,24 @@ private[spark] class BlockManager( logDebug("Block " + blockId + " not found in memory") } } + + // Look for the block in Tachyon + if (level.useOffHeap) { + logDebug("Getting block " + blockId + " from tachyon") + if (tachyonStore.contains(blockId)) { + tachyonStore.getBytes(blockId) match { + case Some(bytes) => { + if (!asValues) { + return Some(bytes) + } else { + return Some(dataDeserialize(blockId, bytes)) + } + } + case None => + logDebug("Block " + blockId + " not found in tachyon") + } + } + } // Look for block on disk, potentially storing it back into memory if required: if (level.useDisk) { @@ -634,6 +668,23 @@ private[spark] class BlockManager( } // Keep track of which blocks are dropped from memory res.droppedBlocks.foreach { block => updatedBlocks += block } + } else if (level.useOffHeap) { + // Save to Tachyon. + val res = data match { + case IteratorValues(iterator) => + tachyonStore.putValues(blockId, iterator, level, false) + case ArrayBufferValues(array) => + tachyonStore.putValues(blockId, array, level, false) + case ByteBufferValues(bytes) => { + bytes.rewind(); + tachyonStore.putBytes(blockId, bytes, level) + } + } + size = res.size + res.data match { + case Right(newBytes) => bytesAfterPut = newBytes + case _ => + } } else { // Save directly to disk. // Don't get back the bytes unless we replicate them. @@ -658,8 +709,8 @@ private[spark] class BlockManager( val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) if (putBlockStatus.storageLevel != StorageLevel.NONE) { - // Now that the block is in either the memory or disk store, let other threads read it, - // and tell the master about it. + // Now that the block is in either the memory, tachyon, or disk store, + // let other threads read it, and tell the master about it. marked = true putBlockInfo.markReady(size) if (tellMaster) { @@ -721,7 +772,8 @@ private[spark] class BlockManager( */ var cachedPeers: Seq[BlockManagerId] = null private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel) { - val tLevel = StorageLevel(level.useDisk, level.useMemory, level.deserialized, 1) + val tLevel = StorageLevel( + level.useDisk, level.useMemory, level.useOffHeap, level.deserialized, 1) if (cachedPeers == null) { cachedPeers = master.getPeers(blockManagerId, level.replication - 1) } @@ -857,9 +909,10 @@ private[spark] class BlockManager( // Removals are idempotent in disk store and memory store. At worst, we get a warning. val removedFromMemory = memoryStore.remove(blockId) val removedFromDisk = diskStore.remove(blockId) - if (!removedFromMemory && !removedFromDisk) { + val removedFromTachyon = if (tachyonInitialized) tachyonStore.remove(blockId) else false + if (!removedFromMemory && !removedFromDisk && !removedFromTachyon) { logWarning("Block " + blockId + " could not be removed as it was not found in either " + - "the disk or memory store") + "the disk, memory, or tachyon store") } blockInfo.remove(blockId) if (tellMaster && info.tellMaster) { @@ -896,6 +949,9 @@ private[spark] class BlockManager( if (level.useDisk) { diskStore.remove(id) } + if (level.useOffHeap) { + tachyonStore.remove(id) + } iterator.remove() logInfo("Dropped block " + id) } @@ -971,6 +1027,9 @@ private[spark] class BlockManager( blockInfo.clear() memoryStore.clear() diskStore.clear() + if (tachyonInitialized) { + tachyonStore.clear() + } metadataCleaner.cancel() broadcastCleaner.cancel() logInfo("BlockManager stopped") diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 497a0f6eb5c1d..7897fade2df2b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -63,9 +63,10 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long): Boolean = { + diskSize: Long, + tachyonSize: Long): Boolean = { val res = askDriverWithReply[Boolean]( - UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize)) + UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize, tachyonSize)) logInfo("Updated info of block " + blockId) res } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 69f261b2002a6..c57b6e8391b13 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -73,10 +73,11 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus register(blockManagerId, maxMemSize, slaveActor) sender ! true - case UpdateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) => + case UpdateBlockInfo( + blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) => // TODO: Ideally we want to handle all the message replies in receive instead of in the // individual private methods. - updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) + updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) case GetLocations(blockId) => sender ! getLocations(blockId) @@ -345,7 +346,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long) { + diskSize: Long, + tachyonSize: Long) { if (!blockManagerInfo.contains(blockManagerId)) { if (blockManagerId.executorId == "" && !isLocal) { @@ -364,7 +366,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus return } - blockManagerInfo(blockManagerId).updateBlockInfo(blockId, storageLevel, memSize, diskSize) + blockManagerInfo(blockManagerId).updateBlockInfo( + blockId, storageLevel, memSize, diskSize, tachyonSize) var locations: mutable.HashSet[BlockManagerId] = null if (blockLocations.containsKey(blockId)) { @@ -408,6 +411,12 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus } } +private[spark] case class BlockStatus( + storageLevel: StorageLevel, + memSize: Long, + diskSize: Long, + tachyonSize: Long) + private[spark] class BlockManagerInfo( val blockManagerId: BlockManagerId, timeMs: Long, @@ -434,7 +443,8 @@ private[spark] class BlockManagerInfo( blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long) { + diskSize: Long, + tachyonSize: Long) { updateLastSeenMs() @@ -448,23 +458,29 @@ private[spark] class BlockManagerInfo( } if (storageLevel.isValid) { - /* isValid means it is either stored in-memory or on-disk. + /* isValid means it is either stored in-memory, on-disk or on-Tachyon. * But the memSize here indicates the data size in or dropped from memory, + * tachyonSize here indicates the data size in or dropped from Tachyon, * and the diskSize here indicates the data size in or dropped to disk. * They can be both larger than 0, when a block is dropped from memory to disk. * Therefore, a safe way to set BlockStatus is to set its info in accurate modes. */ if (storageLevel.useMemory) { - _blocks.put(blockId, BlockStatus(storageLevel, memSize, 0)) + _blocks.put(blockId, BlockStatus(storageLevel, memSize, 0, 0)) _remainingMem -= memSize logInfo("Added %s in memory on %s (size: %s, free: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(memSize), Utils.bytesToString(_remainingMem))) } if (storageLevel.useDisk) { - _blocks.put(blockId, BlockStatus(storageLevel, 0, diskSize)) + _blocks.put(blockId, BlockStatus(storageLevel, 0, diskSize, 0)) logInfo("Added %s on disk on %s (size: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize))) } + if (storageLevel.useOffHeap) { + _blocks.put(blockId, BlockStatus(storageLevel, 0, 0, tachyonSize)) + logInfo("Added %s on tachyon on %s (size: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(tachyonSize))) + } } else if (_blocks.containsKey(blockId)) { // If isValid is not true, drop the block. val blockStatus: BlockStatus = _blocks.get(blockId) @@ -479,6 +495,10 @@ private[spark] class BlockManagerInfo( logInfo("Removed %s on %s on disk (size: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.diskSize))) } + if (blockStatus.storageLevel.useOffHeap) { + logInfo("Removed %s on %s on tachyon (size: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.tachyonSize))) + } } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 365e3900731dc..2b53bf33b5fba 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -60,10 +60,12 @@ private[storage] object BlockManagerMessages { var blockId: BlockId, var storageLevel: StorageLevel, var memSize: Long, - var diskSize: Long) - extends ToBlockManagerMaster with Externalizable { + var diskSize: Long, + var tachyonSize: Long) + extends ToBlockManagerMaster + with Externalizable { - def this() = this(null, null, null, 0, 0) // For deserialization only + def this() = this(null, null, null, 0, 0, 0) // For deserialization only override def writeExternal(out: ObjectOutput) { blockManagerId.writeExternal(out) @@ -71,6 +73,7 @@ private[storage] object BlockManagerMessages { storageLevel.writeExternal(out) out.writeLong(memSize) out.writeLong(diskSize) + out.writeLong(tachyonSize) } override def readExternal(in: ObjectInput) { @@ -79,6 +82,7 @@ private[storage] object BlockManagerMessages { storageLevel = StorageLevel(in) memSize = in.readLong() diskSize = in.readLong() + tachyonSize = in.readLong() } } @@ -88,13 +92,15 @@ private[storage] object BlockManagerMessages { blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long): UpdateBlockInfo = { - new UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize) + diskSize: Long, + tachyonSize: Long): UpdateBlockInfo = { + new UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize, tachyonSize) } // For pattern-matching - def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, BlockId, StorageLevel, Long, Long)] = { - Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize)) + def unapply(h: UpdateBlockInfo) + : Option[(BlockManagerId, BlockId, StorageLevel, Long, Long, Long)] = { + Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize, h.tachyonSize)) } } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 4212a539dab4b..95e71de2d3f1d 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -21,8 +21,9 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} /** * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, - * whether to drop the RDD to disk if it falls out of memory, whether to keep the data in memory - * in a serialized format, and whether to replicate the RDD partitions on multiple nodes. + * or Tachyon, whether to drop the RDD to disk if it falls out of memory or Tachyon , whether to + * keep the data in memory in a serialized format, and whether to replicate the RDD partitions on + * multiple nodes. * The [[org.apache.spark.storage.StorageLevel$]] singleton object contains some static constants * for commonly useful storage levels. To create your own storage level object, use the * factory method of the singleton object (`StorageLevel(...)`). @@ -30,45 +31,58 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} class StorageLevel private( private var useDisk_ : Boolean, private var useMemory_ : Boolean, + private var useOffHeap_ : Boolean, private var deserialized_ : Boolean, private var replication_ : Int = 1) extends Externalizable { // TODO: Also add fields for caching priority, dataset ID, and flushing. private def this(flags: Int, replication: Int) { - this((flags & 4) != 0, (flags & 2) != 0, (flags & 1) != 0, replication) + this((flags & 8) != 0, (flags & 4) != 0, (flags & 2) != 0, (flags & 1) != 0, replication) } - def this() = this(false, true, false) // For deserialization + def this() = this(false, true, false, false) // For deserialization def useDisk = useDisk_ def useMemory = useMemory_ + def useOffHeap = useOffHeap_ def deserialized = deserialized_ def replication = replication_ assert(replication < 40, "Replication restricted to be less than 40 for calculating hashcodes") + if (useOffHeap) { + require(useDisk == false, "Off-heap storage level does not support using disk") + require(useMemory == false, "Off-heap storage level does not support using heap memory") + require(deserialized == false, "Off-heap storage level does not support deserialized storage") + require(replication == 1, "Off-heap storage level does not support multiple replication") + } + override def clone(): StorageLevel = new StorageLevel( - this.useDisk, this.useMemory, this.deserialized, this.replication) + this.useDisk, this.useMemory, this.useOffHeap, this.deserialized, this.replication) override def equals(other: Any): Boolean = other match { case s: StorageLevel => s.useDisk == useDisk && s.useMemory == useMemory && + s.useOffHeap == useOffHeap && s.deserialized == deserialized && s.replication == replication case _ => false } - def isValid = ((useMemory || useDisk) && (replication > 0)) + def isValid = ((useMemory || useDisk || useOffHeap) && (replication > 0)) def toInt: Int = { var ret = 0 if (useDisk_) { - ret |= 4 + ret |= 8 } if (useMemory_) { + ret |= 4 + } + if (useOffHeap_) { ret |= 2 } if (deserialized_) { @@ -84,8 +98,9 @@ class StorageLevel private( override def readExternal(in: ObjectInput) { val flags = in.readByte() - useDisk_ = (flags & 4) != 0 - useMemory_ = (flags & 2) != 0 + useDisk_ = (flags & 8) != 0 + useMemory_ = (flags & 4) != 0 + useOffHeap_ = (flags & 2) != 0 deserialized_ = (flags & 1) != 0 replication_ = in.readByte() } @@ -93,14 +108,15 @@ class StorageLevel private( @throws(classOf[IOException]) private def readResolve(): Object = StorageLevel.getCachedStorageLevel(this) - override def toString: String = - "StorageLevel(%b, %b, %b, %d)".format(useDisk, useMemory, deserialized, replication) + override def toString: String = "StorageLevel(%b, %b, %b, %b, %d)".format( + useDisk, useMemory, useOffHeap, deserialized, replication) override def hashCode(): Int = toInt * 41 + replication def description : String = { var result = "" result += (if (useDisk) "Disk " else "") result += (if (useMemory) "Memory " else "") + result += (if (useOffHeap) "Tachyon " else "") result += (if (deserialized) "Deserialized " else "Serialized ") result += "%sx Replicated".format(replication) result @@ -113,22 +129,28 @@ class StorageLevel private( * new storage levels. */ object StorageLevel { - val NONE = new StorageLevel(false, false, false) - val DISK_ONLY = new StorageLevel(true, false, false) - val DISK_ONLY_2 = new StorageLevel(true, false, false, 2) - val MEMORY_ONLY = new StorageLevel(false, true, true) - val MEMORY_ONLY_2 = new StorageLevel(false, true, true, 2) - val MEMORY_ONLY_SER = new StorageLevel(false, true, false) - val MEMORY_ONLY_SER_2 = new StorageLevel(false, true, false, 2) - val MEMORY_AND_DISK = new StorageLevel(true, true, true) - val MEMORY_AND_DISK_2 = new StorageLevel(true, true, true, 2) - val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false) - val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, 2) + val NONE = new StorageLevel(false, false, false, false) + val DISK_ONLY = new StorageLevel(true, false, false, false) + val DISK_ONLY_2 = new StorageLevel(true, false, false, false, 2) + val MEMORY_ONLY = new StorageLevel(false, true, false, true) + val MEMORY_ONLY_2 = new StorageLevel(false, true, false, true, 2) + val MEMORY_ONLY_SER = new StorageLevel(false, true, false, false) + val MEMORY_ONLY_SER_2 = new StorageLevel(false, true, false, false, 2) + val MEMORY_AND_DISK = new StorageLevel(true, true, false, true) + val MEMORY_AND_DISK_2 = new StorageLevel(true, true, false, true, 2) + val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, false) + val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) + val OFF_HEAP = new StorageLevel(false, false, true, false) + + /** Create a new StorageLevel object without setting useOffHeap */ + def apply(useDisk: Boolean, useMemory: Boolean, useOffHeap: Boolean, + deserialized: Boolean, replication: Int) = getCachedStorageLevel( + new StorageLevel(useDisk, useMemory, useOffHeap, deserialized, replication)) /** Create a new StorageLevel object */ - def apply(useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, - replication: Int = 1): StorageLevel = - getCachedStorageLevel(new StorageLevel(useDisk, useMemory, deserialized, replication)) + def apply(useDisk: Boolean, useMemory: Boolean, + deserialized: Boolean, replication: Int = 1) = getCachedStorageLevel( + new StorageLevel(useDisk, useMemory, false, deserialized, replication)) /** Create a new StorageLevel object from its integer representation */ def apply(flags: Int, replication: Int): StorageLevel = diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index 26565f56ad858..7a174959037be 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -44,7 +44,7 @@ private[spark] class StorageStatusListener extends SparkListener { storageStatusList.foreach { storageStatus => val unpersistedBlocksIds = storageStatus.rddBlocks.keys.filter(_.rddId == unpersistedRDDId) unpersistedBlocksIds.foreach { blockId => - storageStatus.blocks(blockId) = BlockStatus(StorageLevel.NONE, 0L, 0L) + storageStatus.blocks(blockId) = BlockStatus(StorageLevel.NONE, 0L, 0L, 0L) } } } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 6153dfe0b7e13..ff6e84cf9819a 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -48,17 +48,23 @@ class StorageStatus( } private[spark] -class RDDInfo(val id: Int, val name: String, val numPartitions: Int, val storageLevel: StorageLevel) - extends Ordered[RDDInfo] { +class RDDInfo( + val id: Int, + val name: String, + val numPartitions: Int, + val storageLevel: StorageLevel) extends Ordered[RDDInfo] { var numCachedPartitions = 0 var memSize = 0L var diskSize = 0L + var tachyonSize= 0L override def toString = { - ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; " + - "DiskSize: %s").format(name, id, storageLevel.toString, numCachedPartitions, - numPartitions, Utils.bytesToString(memSize), Utils.bytesToString(diskSize)) + import Utils.bytesToString + ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s;" + + "TachyonSize: %s; DiskSize: %s").format( + name, id, storageLevel.toString, numCachedPartitions, numPartitions, + bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize)) } override def compare(that: RDDInfo) = { @@ -105,14 +111,17 @@ object StorageUtils { val rddInfoMap = rddInfos.map { info => (info.id, info) }.toMap val rddStorageInfos = blockStatusMap.flatMap { case (rddId, blocks) => - // Add up memory and disk sizes - val persistedBlocks = blocks.filter { status => status.memSize + status.diskSize > 0 } + // Add up memory, disk and Tachyon sizes + val persistedBlocks = + blocks.filter { status => status.memSize + status.diskSize + status.tachyonSize > 0 } val memSize = persistedBlocks.map(_.memSize).reduceOption(_ + _).getOrElse(0L) val diskSize = persistedBlocks.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + val tachyonSize = persistedBlocks.map(_.tachyonSize).reduceOption(_ + _).getOrElse(0L) rddInfoMap.get(rddId).map { rddInfo => rddInfo.numCachedPartitions = persistedBlocks.length rddInfo.memSize = memSize rddInfo.diskSize = diskSize + rddInfo.tachyonSize = tachyonSize rddInfo } }.toArray diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala new file mode 100644 index 0000000000000..b0b9674856568 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -0,0 +1,155 @@ +/* + * 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.storage + +import java.text.SimpleDateFormat +import java.util.{Date, Random} + +import tachyon.client.TachyonFS +import tachyon.client.TachyonFile + +import org.apache.spark.Logging +import org.apache.spark.executor.ExecutorExitCode +import org.apache.spark.network.netty.ShuffleSender +import org.apache.spark.util.Utils + + +/** + * Creates and maintains the logical mapping between logical blocks and tachyon fs locations. By + * default, one block is mapped to one file with a name given by its BlockId. + * + * @param rootDirs The directories to use for storing block files. Data will be hashed among these. + */ +private[spark] class TachyonBlockManager( + shuffleManager: ShuffleBlockManager, + rootDirs: String, + val master: String) + extends Logging { + + val client = if (master != null && master != "") TachyonFS.get(master) else null + + if (client == null) { + logError("Failed to connect to the Tachyon as the master address is not configured") + System.exit(ExecutorExitCode.TACHYON_STORE_FAILED_TO_INITIALIZE) + } + + private val MAX_DIR_CREATION_ATTEMPTS = 10 + private val subDirsPerTachyonDir = + shuffleManager.conf.get("spark.tachyonStore.subDirectories", "64").toInt + + // Create one Tachyon directory for each path mentioned in spark.tachyonStore.folderName; + // then, inside this directory, create multiple subdirectories that we will hash files into, + // in order to avoid having really large inodes at the top level in Tachyon. + private val tachyonDirs: Array[TachyonFile] = createTachyonDirs() + private val subDirs = Array.fill(tachyonDirs.length)(new Array[TachyonFile](subDirsPerTachyonDir)) + + addShutdownHook() + + def removeFile(file: TachyonFile): Boolean = { + client.delete(file.getPath(), false) + } + + def fileExists(file: TachyonFile): Boolean = { + client.exist(file.getPath()) + } + + def getFile(filename: String): TachyonFile = { + // Figure out which tachyon directory it hashes to, and which subdirectory in that + val hash = Utils.nonNegativeHash(filename) + val dirId = hash % tachyonDirs.length + val subDirId = (hash / tachyonDirs.length) % subDirsPerTachyonDir + + // Create the subdirectory if it doesn't already exist + var subDir = subDirs(dirId)(subDirId) + if (subDir == null) { + subDir = subDirs(dirId).synchronized { + val old = subDirs(dirId)(subDirId) + if (old != null) { + old + } else { + val path = tachyonDirs(dirId) + "/" + "%02x".format(subDirId) + client.mkdir(path) + val newDir = client.getFile(path) + subDirs(dirId)(subDirId) = newDir + newDir + } + } + } + val filePath = subDir + "/" + filename + if(!client.exist(filePath)) { + client.createFile(filePath) + } + val file = client.getFile(filePath) + file + } + + def getFile(blockId: BlockId): TachyonFile = getFile(blockId.name) + + // TODO: Some of the logic here could be consolidated/de-duplicated with that in the DiskStore. + private def createTachyonDirs(): Array[TachyonFile] = { + logDebug("Creating tachyon directories at root dirs '" + rootDirs + "'") + val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") + rootDirs.split(",").map { rootDir => + var foundLocalDir = false + var tachyonDir: TachyonFile = null + var tachyonDirId: String = null + var tries = 0 + val rand = new Random() + while (!foundLocalDir && tries < MAX_DIR_CREATION_ATTEMPTS) { + tries += 1 + try { + tachyonDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536)) + val path = rootDir + "/" + "spark-tachyon-" + tachyonDirId + if (!client.exist(path)) { + foundLocalDir = client.mkdir(path) + tachyonDir = client.getFile(path) + } + } catch { + case e: Exception => + logWarning("Attempt " + tries + " to create tachyon dir " + tachyonDir + " failed", e) + } + } + if (!foundLocalDir) { + logError("Failed " + MAX_DIR_CREATION_ATTEMPTS + " attempts to create tachyon dir in " + + rootDir) + System.exit(ExecutorExitCode.TACHYON_STORE_FAILED_TO_CREATE_DIR) + } + logInfo("Created tachyon directory at " + tachyonDir) + tachyonDir + } + } + + private def addShutdownHook() { + tachyonDirs.foreach(tachyonDir => Utils.registerShutdownDeleteDir(tachyonDir)) + Runtime.getRuntime.addShutdownHook(new Thread("delete Spark tachyon dirs") { + override def run() { + logDebug("Shutdown hook called") + tachyonDirs.foreach { tachyonDir => + try { + if (!Utils.hasRootAsShutdownDeleteDir(tachyonDir)) { + Utils.deleteRecursively(tachyonDir, client) + } + } catch { + case t: Throwable => + logError("Exception while deleting tachyon spark dir: " + tachyonDir, t) + } + } + } + }) + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala b/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala new file mode 100644 index 0000000000000..b86abbda1d3e7 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala @@ -0,0 +1,28 @@ +/* + * 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.storage + +import tachyon.client.TachyonFile + +/** + * References a particular segment of a file (potentially the entire file), based off an offset and + * a length. + */ +private[spark] class TachyonFileSegment(val file: TachyonFile, val offset: Long, val length: Long) { + override def toString = "(name=%s, offset=%d, length=%d)".format(file.getPath(), offset, length) +} diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala new file mode 100644 index 0000000000000..c37e76f893605 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -0,0 +1,142 @@ +/* + * 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.storage + +import java.io.IOException +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer + +import tachyon.client.{WriteType, ReadType} + +import org.apache.spark.Logging +import org.apache.spark.util.Utils +import org.apache.spark.serializer.Serializer + + +private class Entry(val size: Long) + + +/** + * Stores BlockManager blocks on Tachyon. + */ +private class TachyonStore( + blockManager: BlockManager, + tachyonManager: TachyonBlockManager) + extends BlockStore(blockManager: BlockManager) with Logging { + + logInfo("TachyonStore started") + + override def getSize(blockId: BlockId): Long = { + tachyonManager.getFile(blockId.name).length + } + + override def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): PutResult = { + putToTachyonStore(blockId, bytes, true) + } + + override def putValues( + blockId: BlockId, + values: ArrayBuffer[Any], + level: StorageLevel, + returnValues: Boolean): PutResult = { + return putValues(blockId, values.toIterator, level, returnValues) + } + + override def putValues( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + returnValues: Boolean): PutResult = { + logDebug("Attempting to write values for block " + blockId) + val _bytes = blockManager.dataSerialize(blockId, values) + putToTachyonStore(blockId, _bytes, returnValues) + } + + private def putToTachyonStore( + blockId: BlockId, + bytes: ByteBuffer, + returnValues: Boolean): PutResult = { + // So that we do not modify the input offsets ! + // duplicate does not copy buffer, so inexpensive + val byteBuffer = bytes.duplicate() + byteBuffer.rewind() + logDebug("Attempting to put block " + blockId + " into Tachyon") + val startTime = System.currentTimeMillis + val file = tachyonManager.getFile(blockId) + val os = file.getOutStream(WriteType.TRY_CACHE) + os.write(byteBuffer.array()) + os.close() + val finishTime = System.currentTimeMillis + logDebug("Block %s stored as %s file in Tachyon in %d ms".format( + blockId, Utils.bytesToString(byteBuffer.limit), (finishTime - startTime))) + + if (returnValues) { + PutResult(bytes.limit(), Right(bytes.duplicate())) + } else { + PutResult(bytes.limit(), null) + } + } + + override def remove(blockId: BlockId): Boolean = { + val file = tachyonManager.getFile(blockId) + if (tachyonManager.fileExists(file)) { + tachyonManager.removeFile(file) + } else { + false + } + } + + override def getValues(blockId: BlockId): Option[Iterator[Any]] = { + getBytes(blockId).map(buffer => blockManager.dataDeserialize(blockId, buffer)) + } + + + override def getBytes(blockId: BlockId): Option[ByteBuffer] = { + val file = tachyonManager.getFile(blockId) + if (file == null || file.getLocationHosts().size == 0) { + return None + } + val is = file.getInStream(ReadType.CACHE) + var buffer: ByteBuffer = null + try { + if (is != null) { + val size = file.length + val bs = new Array[Byte](size.asInstanceOf[Int]) + val fetchSize = is.read(bs, 0, size.asInstanceOf[Int]) + buffer = ByteBuffer.wrap(bs) + if (fetchSize != size) { + logWarning("Failed to fetch the block " + blockId + " from Tachyon : Size " + size + + " is not equal to fetched size " + fetchSize) + return None + } + } + } catch { + case ioe: IOException => { + logWarning("Failed to fetch the block " + blockId + " from Tachyon", ioe) + return None + } + } + Some(buffer) + } + + override def contains(blockId: BlockId): Boolean = { + val file = tachyonManager.getFile(blockId) + tachyonManager.fileExists(file) + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index d10aa12b9ebca..048f671c8788f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -81,9 +81,8 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { /** If stages is too large, remove and garbage collect old stages */ private def trimIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { if (stages.size > retainedStages) { - val toRemove = retainedStages / 10 - stages.takeRight(toRemove).foreach( s => { - stageIdToTaskData.remove(s.stageId) + val toRemove = math.max(retainedStages / 10, 1) + stages.take(toRemove).foreach { s => stageIdToTime.remove(s.stageId) stageIdToShuffleRead.remove(s.stageId) stageIdToShuffleWrite.remove(s.stageId) @@ -92,10 +91,12 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { stageIdToTasksActive.remove(s.stageId) stageIdToTasksComplete.remove(s.stageId) stageIdToTasksFailed.remove(s.stageId) + stageIdToTaskData.remove(s.stageId) + stageIdToExecutorSummaries.remove(s.stageId) stageIdToPool.remove(s.stageId) - if (stageIdToDescription.contains(s.stageId)) {stageIdToDescription.remove(s.stageId)} - }) - stages.trimEnd(toRemove) + stageIdToDescription.remove(s.stageId) + } + stages.trimStart(toRemove) } } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index b2732de51058a..0fa461e5e9d27 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -33,6 +33,7 @@ private[ui] class IndexPage(parent: BlockManagerUI) { private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { + val rdds = listener.rddInfoList val content = UIUtils.listingTable(rddHeader, rddRow, rdds) UIUtils.headerSparkPage(content, basePath, appName, "Storage ", Storage) @@ -45,6 +46,7 @@ private[ui] class IndexPage(parent: BlockManagerUI) { "Cached Partitions", "Fraction Cached", "Size in Memory", + "Size in Tachyon", "Size on Disk") /** Render an HTML row representing an RDD */ @@ -60,6 +62,7 @@ private[ui] class IndexPage(parent: BlockManagerUI) { {rdd.numCachedPartitions} {"%.0f%%".format(rdd.numCachedPartitions * 100.0 / rdd.numPartitions)} {Utils.bytesToString(rdd.memSize)} + {Utils.bytesToString(rdd.tachyonSize)} {Utils.bytesToString(rdd.diskSize)} } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index d9a6af61872d1..2155a8888c85c 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -274,12 +274,14 @@ private[spark] object JsonProtocol { ("Number of Partitions" -> rddInfo.numPartitions) ~ ("Number of Cached Partitions" -> rddInfo.numCachedPartitions) ~ ("Memory Size" -> rddInfo.memSize) ~ + ("Tachyon Size" -> rddInfo.tachyonSize) ~ ("Disk Size" -> rddInfo.diskSize) } def storageLevelToJson(storageLevel: StorageLevel): JValue = { ("Use Disk" -> storageLevel.useDisk) ~ ("Use Memory" -> storageLevel.useMemory) ~ + ("Use Tachyon" -> storageLevel.useOffHeap) ~ ("Deserialized" -> storageLevel.deserialized) ~ ("Replication" -> storageLevel.replication) } @@ -288,6 +290,7 @@ private[spark] object JsonProtocol { val storageLevel = storageLevelToJson(blockStatus.storageLevel) ("Storage Level" -> storageLevel) ~ ("Memory Size" -> blockStatus.memSize) ~ + ("Tachyon Size" -> blockStatus.tachyonSize) ~ ("Disk Size" -> blockStatus.diskSize) } @@ -570,11 +573,13 @@ private[spark] object JsonProtocol { val numPartitions = (json \ "Number of Partitions").extract[Int] val numCachedPartitions = (json \ "Number of Cached Partitions").extract[Int] val memSize = (json \ "Memory Size").extract[Long] + val tachyonSize = (json \ "Tachyon Size").extract[Long] val diskSize = (json \ "Disk Size").extract[Long] val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel) rddInfo.numCachedPartitions = numCachedPartitions rddInfo.memSize = memSize + rddInfo.tachyonSize = tachyonSize rddInfo.diskSize = diskSize rddInfo } @@ -582,16 +587,18 @@ private[spark] object JsonProtocol { def storageLevelFromJson(json: JValue): StorageLevel = { val useDisk = (json \ "Use Disk").extract[Boolean] val useMemory = (json \ "Use Memory").extract[Boolean] + val useTachyon = (json \ "Use Tachyon").extract[Boolean] val deserialized = (json \ "Deserialized").extract[Boolean] val replication = (json \ "Replication").extract[Int] - StorageLevel(useDisk, useMemory, deserialized, replication) + StorageLevel(useDisk, useMemory, useTachyon, deserialized, replication) } def blockStatusFromJson(json: JValue): BlockStatus = { val storageLevel = storageLevelFromJson(json \ "Storage Level") val memorySize = (json \ "Memory Size").extract[Long] val diskSize = (json \ "Disk Size").extract[Long] - BlockStatus(storageLevel, memorySize, diskSize) + val tachyonSize = (json \ "Tachyon Size").extract[Long] + BlockStatus(storageLevel, memorySize, diskSize, tachyonSize) } 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 0b07bdcf63b97..59da51f3e0297 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -26,6 +26,7 @@ import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.SortedSet import scala.io.Source import scala.reflect.ClassTag @@ -33,16 +34,20 @@ import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.json4s._ +import tachyon.client.{TachyonFile,TachyonFS} import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} + /** * Various utility methods used by Spark. */ private[spark] object Utils extends Logging { + val osName = System.getProperty("os.name") + /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -150,6 +155,7 @@ private[spark] object Utils extends Logging { } private val shutdownDeletePaths = new scala.collection.mutable.HashSet[String]() + private val shutdownDeleteTachyonPaths = new scala.collection.mutable.HashSet[String]() // Register the path to be deleted via shutdown hook def registerShutdownDeleteDir(file: File) { @@ -159,6 +165,14 @@ private[spark] object Utils extends Logging { } } + // Register the tachyon path to be deleted via shutdown hook + def registerShutdownDeleteDir(tachyonfile: TachyonFile) { + val absolutePath = tachyonfile.getPath() + shutdownDeleteTachyonPaths.synchronized { + shutdownDeleteTachyonPaths += absolutePath + } + } + // Is the path already registered to be deleted via a shutdown hook ? def hasShutdownDeleteDir(file: File): Boolean = { val absolutePath = file.getAbsolutePath() @@ -167,6 +181,14 @@ private[spark] object Utils extends Logging { } } + // Is the path already registered to be deleted via a shutdown hook ? + def hasShutdownDeleteTachyonDir(file: TachyonFile): Boolean = { + val absolutePath = file.getPath() + shutdownDeletePaths.synchronized { + shutdownDeletePaths.contains(absolutePath) + } + } + // Note: if file is child of some registered path, while not equal to it, then return true; // else false. This is to ensure that two shutdown hooks do not try to delete each others // paths - resulting in IOException and incomplete cleanup. @@ -183,6 +205,22 @@ private[spark] object Utils extends Logging { retval } + // Note: if file is child of some registered path, while not equal to it, then return true; + // else false. This is to ensure that two shutdown hooks do not try to delete each others + // paths - resulting in Exception and incomplete cleanup. + def hasRootAsShutdownDeleteDir(file: TachyonFile): Boolean = { + val absolutePath = file.getPath() + val retval = shutdownDeletePaths.synchronized { + shutdownDeletePaths.find { path => + !absolutePath.equals(path) && absolutePath.startsWith(path) + }.isDefined + } + if (retval) { + logInfo("path = " + file + ", already present as root for deletion.") + } + retval + } + /** Create a temporary directory inside the given parent directory */ def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { var attempts = 0 @@ -521,9 +559,10 @@ private[spark] object Utils extends Logging { /** * Delete a file or directory and its contents recursively. + * Don't follow directories if they are symlinks. */ def deleteRecursively(file: File) { - if (file.isDirectory) { + if ((file.isDirectory) && !isSymlink(file)) { for (child <- listFilesSafely(file)) { deleteRecursively(child) } @@ -536,6 +575,49 @@ private[spark] object Utils extends Logging { } } + /** + * Delete a file or directory and its contents recursively. + */ + def deleteRecursively(dir: TachyonFile, client: TachyonFS) { + if (!client.delete(dir.getPath(), true)) { + throw new IOException("Failed to delete the tachyon dir: " + dir) + } + } + + /** + * Check to see if file is a symbolic link. + */ + def isSymlink(file: File): Boolean = { + if (file == null) throw new NullPointerException("File must not be null") + if (osName.startsWith("Windows")) return false + val fileInCanonicalDir = if (file.getParent() == null) { + file + } else { + new File(file.getParentFile().getCanonicalFile(), file.getName()) + } + + if (fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile())) { + return false + } else { + return true + } + } + + /** + * Finds all the files in a directory whose last modified time is older than cutoff seconds. + * @param dir must be the path to a directory, or IllegalArgumentException is thrown + * @param cutoff measured in seconds. Files older than this are returned. + */ + def findOldFiles(dir: File, cutoff: Long): Seq[File] = { + val currentTimeMillis = System.currentTimeMillis + if (dir.isDirectory) { + val files = listFilesSafely(dir) + files.filter { file => file.lastModified < (currentTimeMillis - cutoff * 1000) } + } else { + throw new IllegalArgumentException(dir + " is not a directory!") + } + } + /** * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of megabytes. */ @@ -898,6 +980,26 @@ private[spark] object Utils extends Logging { count } + /** + * Creates a symlink. Note jdk1.7 has Files.createSymbolicLink but not used here + * for jdk1.6 support. Supports windows by doing copy, everything else uses "ln -sf". + * @param src absolute path to the source + * @param dst relative path for the destination + */ + def symlink(src: File, dst: File) { + if (!src.isAbsolute()) { + throw new IOException("Source must be absolute") + } + if (dst.isAbsolute()) { + throw new IOException("Destination must be relative") + } + val linkCmd = if (osName.startsWith("Windows")) "copy" else "ln -sf" + import scala.sys.process._ + (linkCmd + " " + src.getAbsolutePath() + " " + dst.getPath()) lines_! ProcessLogger(line => + (logInfo(line))) + } + + /** Return the class name of the given object, removing all dollar signs */ def getFormattedClassName(obj: AnyRef) = { obj.getClass.getSimpleName.replace("$", "") diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index c6b65c7348ae0..762405be2a8f9 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -17,13 +17,12 @@ package org.apache.spark; -import java.io.File; -import java.io.IOException; -import java.io.Serializable; +import java.io.*; import java.util.*; import scala.Tuple2; +import com.google.common.collect.Lists; import com.google.common.base.Optional; import com.google.common.base.Charsets; import com.google.common.io.Files; @@ -181,6 +180,14 @@ public void call(String s) { Assert.assertEquals(2, foreachCalls); } + @Test + public void toLocalIterator() { + List correct = Arrays.asList(1, 2, 3, 4); + JavaRDD rdd = sc.parallelize(correct); + List result = Lists.newArrayList(rdd.toLocalIterator()); + Assert.assertTrue(correct.equals(result)); + } + @SuppressWarnings("unchecked") @Test public void lookup() { @@ -599,6 +606,32 @@ public void textFiles() throws IOException { Assert.assertEquals(expected, readRDD.collect()); } + @Test + public void wholeTextFiles() throws IOException { + byte[] content1 = "spark is easy to use.\n".getBytes(); + byte[] content2 = "spark is also easy to use.\n".getBytes(); + + File tempDir = Files.createTempDir(); + String tempDirName = tempDir.getAbsolutePath(); + DataOutputStream ds = new DataOutputStream(new FileOutputStream(tempDirName + "/part-00000")); + ds.write(content1); + ds.close(); + ds = new DataOutputStream(new FileOutputStream(tempDirName + "/part-00001")); + ds.write(content2); + ds.close(); + + HashMap container = new HashMap(); + container.put(tempDirName+"/part-00000", new Text(content1).toString()); + container.put(tempDirName+"/part-00001", new Text(content2).toString()); + + JavaPairRDD readRDD = sc.wholeTextFiles(tempDirName); + List> result = readRDD.collect(); + + for (Tuple2 res : result) { + Assert.assertEquals(res._2(), container.get(res._1())); + } + } + @Test public void textFilesCompressed() throws IOException { File tempDir = Files.createTempDir(); diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index 6e7fd55fa4bb1..627e9b5cd9060 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -17,8 +17,11 @@ package org.apache.spark -import org.scalatest.FunSuite +import java.io.File + +import com.google.common.io.Files +import org.scalatest.FunSuite import org.apache.spark.rdd.{HadoopRDD, PipedRDD, HadoopPartition} import org.apache.hadoop.mapred.{JobConf, TextInputFormat, FileSplit} @@ -126,6 +129,29 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { } } + test("basic pipe with separate working directory") { + if (testCommandAvailable("cat")) { + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + val piped = nums.pipe(Seq("cat"), separateWorkingDir = true) + val c = piped.collect() + assert(c.size === 4) + assert(c(0) === "1") + assert(c(1) === "2") + assert(c(2) === "3") + assert(c(3) === "4") + val pipedPwd = nums.pipe(Seq("pwd"), separateWorkingDir = true) + val collectPwd = pipedPwd.collect() + assert(collectPwd(0).contains("tasks/")) + val pipedLs = nums.pipe(Seq("ls"), separateWorkingDir = true).collect() + // make sure symlinks were created + assert(pipedLs.length > 0) + // clean up top level tasks directory + new File("tasks").delete() + } else { + assert(true) + } + } + test("test pipe exports map_input_file") { testExportInputFile("map_input_file") } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index b543471a5d35b..94fba102865b3 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -51,6 +51,14 @@ class SparkContextSchedulerCreationSuite } } + test("local-*") { + val sched = createTaskScheduler("local[*]") + sched.backend match { + case s: LocalBackend => assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case _ => fail() + } + } + test("local-n") { val sched = createTaskScheduler("local[5]") assert(sched.maxTaskFailures === 1) diff --git a/core/src/test/scala/org/apache/spark/TestUtils.scala b/core/src/test/scala/org/apache/spark/TestUtils.scala new file mode 100644 index 0000000000000..1611d09652d40 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/TestUtils.scala @@ -0,0 +1,98 @@ +/* + * 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 + +import java.io.{File, FileInputStream, FileOutputStream} +import java.net.{URI, URL} +import java.util.jar.{JarEntry, JarOutputStream} + +import scala.collection.JavaConversions._ + +import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} +import com.google.common.io.Files + +object TestUtils { + + /** + * Create a jar that defines classes with the given names. + * + * Note: if this is used during class loader tests, class names should be unique + * in order to avoid interference between tests. + */ + def createJarWithClasses(classNames: Seq[String]): URL = { + val tempDir = Files.createTempDir() + val files = for (name <- classNames) yield createCompiledClass(name, tempDir) + val jarFile = new File(tempDir, "testJar-%s.jar".format(System.currentTimeMillis())) + createJar(files, jarFile) + } + + /** + * Create a jar file that contains this set of files. All files will be located at the root + * of the jar. + */ + def createJar(files: Seq[File], jarFile: File): URL = { + val jarFileStream = new FileOutputStream(jarFile) + val jarStream = new JarOutputStream(jarFileStream, new java.util.jar.Manifest()) + + for (file <- files) { + val jarEntry = new JarEntry(file.getName) + jarStream.putNextEntry(jarEntry) + + val in = new FileInputStream(file) + val buffer = new Array[Byte](10240) + var nRead = 0 + while (nRead <= 0) { + nRead = in.read(buffer, 0, buffer.length) + jarStream.write(buffer, 0, nRead) + } + in.close() + } + jarStream.close() + jarFileStream.close() + + jarFile.toURI.toURL + } + + // Adapted from the JavaCompiler.java doc examples + private val SOURCE = JavaFileObject.Kind.SOURCE + private def createURI(name: String) = { + URI.create(s"string:///${name.replace(".", "/")}${SOURCE.extension}") + } + + private class JavaSourceFromString(val name: String, val code: String) + extends SimpleJavaFileObject(createURI(name), SOURCE) { + override def getCharContent(ignoreEncodingErrors: Boolean) = code + } + + /** Creates a compiled class with the given name. Class file will be placed in destDir. */ + def createCompiledClass(className: String, destDir: File): File = { + val compiler = ToolProvider.getSystemJavaCompiler + val sourceFile = new JavaSourceFromString(className, s"public class $className {}") + + // Calling this outputs a class file in pwd. It's easier to just rename the file than + // build a custom FileManager that controls the output location. + compiler.getTask(null, null, null, null, null, Seq(sourceFile)).call() + + val fileName = className + ".class" + val result = new File(fileName) + if (!result.exists()) throw new Exception("Compiled file not found: " + fileName) + val out = new File(destDir, fileName) + result.renameTo(out) + out + } +} diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala new file mode 100644 index 0000000000000..09e35bfc8f85f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -0,0 +1,105 @@ +/* + * 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.input + +import java.io.DataOutputStream +import java.io.File +import java.io.FileOutputStream + +import scala.collection.immutable.IndexedSeq + +import com.google.common.io.Files + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import org.apache.hadoop.io.Text + +import org.apache.spark.SparkContext + +/** + * Tests the correctness of + * [[org.apache.spark.input.WholeTextFileRecordReader WholeTextFileRecordReader]]. A temporary + * directory is created as fake input. Temporal storage would be deleted in the end. + */ +class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { + private var sc: SparkContext = _ + + override def beforeAll() { + sc = new SparkContext("local", "test") + + // Set the block size of local file system to test whether files are split right or not. + sc.hadoopConfiguration.setLong("fs.local.block.size", 32) + } + + override def afterAll() { + sc.stop() + } + + private def createNativeFile(inputDir: File, fileName: String, contents: Array[Byte]) = { + val out = new DataOutputStream(new FileOutputStream(s"${inputDir.toString}/$fileName")) + out.write(contents, 0, contents.length) + out.close() + } + + /** + * This code will test the behaviors of WholeTextFileRecordReader based on local disk. There are + * three aspects to check: + * 1) Whether all files are read; + * 2) Whether paths are read correctly; + * 3) Does the contents be the same. + */ + test("Correctness of WholeTextFileRecordReader.") { + + val dir = Files.createTempDir() + println(s"Local disk address is ${dir.toString}.") + + WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => + createNativeFile(dir, filename, contents) + } + + val res = sc.wholeTextFiles(dir.toString).collect() + + assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size, + "Number of files read out does not fit with the actual value.") + + for ((filename, contents) <- res) { + val shortName = filename.split('/').last + assert(WholeTextFileRecordReaderSuite.fileNames.contains(shortName), + s"Missing file name $filename.") + assert(contents === new Text(WholeTextFileRecordReaderSuite.files(shortName)).toString, + s"file $filename contents can not match.") + } + + dir.delete() + } +} + +/** + * Files to be tested are defined here. + */ +object WholeTextFileRecordReaderSuite { + private val testWords: IndexedSeq[Byte] = "Spark is easy to use.\n".map(_.toByte) + + private val fileNames = Array("part-00000", "part-00001", "part-00002") + private val fileLengths = Array(10, 100, 1000) + + private val files = fileLengths.zip(fileNames).map { case (upperBound, filename) => + filename -> Stream.continually(testWords.toList.toStream).flatten.take(upperBound).toArray + }.toMap +} diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index d6b5fdc7984b4..25973348a7837 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -33,6 +33,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("basic operations") { val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) assert(nums.collect().toList === List(1, 2, 3, 4)) + assert(nums.toLocalIterator.toList === List(1, 2, 3, 4)) val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2) assert(dups.distinct().count() === 4) assert(dups.distinct.count === 4) // Can distinct and count be called without parentheses? diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 9aaf3601e430e..e10ec7d2624a0 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -97,9 +97,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("StorageLevel object caching") { - val level1 = StorageLevel(false, false, false, 3) - val level2 = StorageLevel(false, false, false, 3) // this should return the same object as level1 - val level3 = StorageLevel(false, false, false, 2) // this should return a different object + val level1 = StorageLevel(false, false, false, false, 3) + val level2 = StorageLevel(false, false, false, false, 3) // this should return the same object as level1 + val level3 = StorageLevel(false, false, false, false, 2) // this should return a different object assert(level2 === level1, "level2 is not same as level1") assert(level2.eq(level1), "level2 is not the same object as level1") assert(level3 != level1, "level3 is same as level1") @@ -494,6 +494,26 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(store.memoryStore.contains(rdd(0, 3)), "rdd_0_3 was not in store") } + test("tachyon storage") { + // TODO Make the spark.test.tachyon.enable true after using tachyon 0.5.0 testing jar. + val tachyonUnitTestEnabled = conf.getBoolean("spark.test.tachyon.enable", false) + if (tachyonUnitTestEnabled) { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) + val a1 = new Array[Byte](400) + val a2 = new Array[Byte](400) + val a3 = new Array[Byte](400) + store.putSingle("a1", a1, StorageLevel.OFF_HEAP) + store.putSingle("a2", a2, StorageLevel.OFF_HEAP) + store.putSingle("a3", a3, StorageLevel.OFF_HEAP) + assert(store.getSingle("a3").isDefined, "a3 was in store") + assert(store.getSingle("a2").isDefined, "a2 was in store") + assert(store.getSingle("a1").isDefined, "a1 was in store") + } else { + info("tachyon storage test disabled.") + } + } + test("on-disk storage") { store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr, mapOutputTracker) diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index d8a3e859f85cd..beac656f573b4 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -18,16 +18,45 @@ package org.apache.spark.ui.jobs import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers -import org.apache.spark.{LocalSparkContext, SparkContext, Success} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, Success} import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} import org.apache.spark.scheduler._ import org.apache.spark.util.Utils -class JobProgressListenerSuite extends FunSuite with LocalSparkContext { +class JobProgressListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers { + test("test LRU eviction of stages") { + val conf = new SparkConf() + conf.set("spark.ui.retainedStages", 5.toString) + val listener = new JobProgressListener(conf) + + def createStageStartEvent(stageId: Int) = { + val stageInfo = new StageInfo(stageId, stageId.toString, 0, null) + SparkListenerStageSubmitted(stageInfo) + } + + def createStageEndEvent(stageId: Int) = { + val stageInfo = new StageInfo(stageId, stageId.toString, 0, null) + SparkListenerStageCompleted(stageInfo) + } + + for (i <- 1 to 50) { + listener.onStageSubmitted(createStageStartEvent(i)) + listener.onStageCompleted(createStageEndEvent(i)) + } + + listener.completedStages.size should be (5) + listener.completedStages.filter(_.stageId == 50).size should be (1) + listener.completedStages.filter(_.stageId == 49).size should be (1) + listener.completedStages.filter(_.stageId == 48).size should be (1) + listener.completedStages.filter(_.stageId == 47).size should be (1) + listener.completedStages.filter(_.stageId == 46).size should be (1) + } + test("test executor id to summary") { - val sc = new SparkContext("local", "test") - val listener = new JobProgressListener(sc.conf) + val conf = new SparkConf() + val listener = new JobProgressListener(conf) val taskMetrics = new TaskMetrics() val shuffleReadMetrics = new ShuffleReadMetrics() diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 79075a7eb847c..7bab7da8fed68 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -455,7 +455,7 @@ class JsonProtocolSuite extends FunSuite { t.shuffleWriteMetrics = Some(sw) // Make at most 6 blocks t.updatedBlocks = Some((1 to (e % 5 + 1)).map { i => - (RDDBlockId(e % i, f % i), BlockStatus(StorageLevel.MEMORY_AND_DISK_SER_2, a % i, b % i)) + (RDDBlockId(e % i, f % i), BlockStatus(StorageLevel.MEMORY_AND_DISK_SER_2, a % i, b % i, c%i)) }.toSeq) t } @@ -469,19 +469,19 @@ class JsonProtocolSuite extends FunSuite { """ {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":100,"Stage Name": "greetings","Number of Tasks":200,"RDD Info":{"RDD ID":100,"Name":"mayor","Storage - Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1}, - "Number of Partitions":200,"Number of Cached Partitions":300,"Memory Size":400, - "Disk Size":500},"Emitted Task Size Warning":false},"Properties":{"France":"Paris", - "Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} + Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":true, + "Replication":1},"Number of Partitions":200,"Number of Cached Partitions":300, + "Memory Size":400,"Disk Size":500,"Tachyon Size":0},"Emitted Task Size Warning":false}, + "Properties":{"France":"Paris","Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} """ private val stageCompletedJsonString = """ {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":101,"Stage Name": "greetings","Number of Tasks":201,"RDD Info":{"RDD ID":101,"Name":"mayor","Storage - Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1}, - "Number of Partitions":201,"Number of Cached Partitions":301,"Memory Size":401, - "Disk Size":501},"Emitted Task Size Warning":false}} + Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":true, + "Replication":1},"Number of Partitions":201,"Number of Cached Partitions":301, + "Memory Size":401,"Disk Size":501,"Tachyon Size":0},"Emitted Task Size Warning":false}} """ private val taskStartJsonString = @@ -514,8 +514,8 @@ class JsonProtocolSuite extends FunSuite { 700,"Fetch Wait Time":900,"Remote Bytes Read":1000},"Shuffle Write Metrics": {"Shuffle Bytes Written":1200,"Shuffle Write Time":1500},"Updated Blocks": [{"Block ID":{"Type":"RDDBlockId","RDD ID":0,"Split Index":0},"Status": - {"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":false, - "Replication":2},"Memory Size":0,"Disk Size":0}}]}} + {"Storage Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":false, + "Replication":2},"Memory Size":0,"Disk Size":0,"Tachyon Size":0}}]}} """ private val jobStartJsonString = diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 616214fb5e3a6..eb7fb6318262b 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.util import scala.util.Random -import java.io.{ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} +import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} import java.nio.{ByteBuffer, ByteOrder} import com.google.common.base.Charsets @@ -154,5 +154,18 @@ class UtilsSuite extends FunSuite { val iterator = Iterator.range(0, 5) assert(Utils.getIteratorSize(iterator) === 5L) } + + test("findOldFiles") { + // create some temporary directories and files + val parent: File = Utils.createTempDir() + val child1: File = Utils.createTempDir(parent.getCanonicalPath) // The parent directory has two child directories + val child2: File = Utils.createTempDir(parent.getCanonicalPath) + // set the last modified time of child1 to 10 secs old + child1.setLastModified(System.currentTimeMillis() - (1000 * 10)) + + val result = Utils.findOldFiles(parent, 5) // find files older than 5 secs + assert(result.size.equals(1)) + assert(result(0).getCanonicalPath.equals(child1.getCanonicalPath)) + } } diff --git a/dev/audit-release/maven_app_core/pom.xml b/dev/audit-release/maven_app_core/pom.xml index 0b837c01751fe..76a381f8e17e0 100644 --- a/dev/audit-release/maven_app_core/pom.xml +++ b/dev/audit-release/maven_app_core/pom.xml @@ -49,7 +49,7 @@ maven-compiler-plugin - 2.3.2 + 3.1 diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 995106f111443..bf1c5d7953bd2 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -49,14 +49,14 @@ mvn -DskipTests \ -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Pspark-ganglia-lgpl \ + -Pyarn -Phive -Pspark-ganglia-lgpl\ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ --batch-mode release:prepare mvn -DskipTests \ -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Pspark-ganglia-lgpl\ release:perform rm -rf spark diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index e8f78fc5f231a..7a61943e94814 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -87,11 +87,20 @@ def merge_pr(pr_num, target_ref): run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, target_ref, target_branch_name)) run_cmd("git checkout %s" % target_branch_name) - run_cmd(['git', 'merge', pr_branch_name, '--squash']) + had_conflicts = False + try: + run_cmd(['git', 'merge', pr_branch_name, '--squash']) + except Exception as e: + msg = "Error merging: %s\nWould you like to manually fix-up this merge?" % e + continue_maybe(msg) + msg = "Okay, please fix any conflicts and 'git add' conflicting files... Finished?" + continue_maybe(msg) + had_conflicts = True commit_authors = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, '--pretty=format:%an <%ae>']).split("\n") - distinct_authors = sorted(set(commit_authors), key=lambda x: commit_authors.count(x), reverse=True) + distinct_authors = sorted(set(commit_authors), key=lambda x: commit_authors.count(x), + reverse=True) primary_author = distinct_authors[0] commits = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, '--pretty=format:%h [%an] %s']).split("\n\n") @@ -105,6 +114,13 @@ def merge_pr(pr_num, target_ref): merge_message_flags += ["-m", authors] + if had_conflicts: + committer_name = run_cmd("git config --get user.name").strip() + committer_email = run_cmd("git config --get user.email").strip() + message = "This patch had conflicts when merged, resolved by\nCommitter: %s <%s>" % ( + committer_name, committer_email) + merge_message_flags += ["-m", message] + # The string "Closes #%s" string is required for GitHub to correctly close the PR merge_message_flags += ["-m", "Closes #%s from %s and squashes the following commits:" % (pr_num, pr_repo_desc)] @@ -186,8 +202,10 @@ def maybe_cherry_pick(pr_num, merge_hash, default_branch): maybe_cherry_pick(pr_num, merge_hash, latest_branch) sys.exit(0) -if bool(pr["mergeable"]) == False: - fail("Pull request %s is not mergeable in its current form" % pr_num) +if not bool(pr["mergeable"]): + msg = "Pull request %s is not mergeable in its current form.\n" % pr_num + \ + "Continue? (experts only!)" + continue_maybe(msg) print ("\n=== Pull Request #%s ===" % pr_num) print("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( diff --git a/dev/run-tests b/dev/run-tests index fff949e04fcd7..6ad674a2ba127 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -26,13 +26,12 @@ rm -rf ./work # Fail fast set -e - +set -o pipefail if test -x "$JAVA_HOME/bin/java"; then declare java_cmd="$JAVA_HOME/bin/java" else declare java_cmd=java fi - JAVA_VERSION=$($java_cmd -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') [ "$JAVA_VERSION" -ge 18 ] && echo "" || echo "[Warn] Java 8 tests will not run because JDK version is < 1.8." @@ -49,7 +48,9 @@ dev/scalastyle echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" -sbt/sbt assembly test +# echo "q" is needed because sbt on encountering a build file with failure (either resolution or compilation) +# prompts the user for input either q, r, etc to quit or retry. This echo is there to make it not block. +echo -e "q\n" | sbt/sbt assembly test | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" echo "=========================================================================" echo "Running PySpark tests" @@ -63,5 +64,4 @@ echo "=========================================================================" echo "Detecting binary incompatibilites with MiMa" echo "=========================================================================" ./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore -sbt/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" - +echo -e "q\n" | sbt/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" diff --git a/dev/scalastyle b/dev/scalastyle index 5a18f4d672825..19955b9aaaad3 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,8 +17,8 @@ # limitations under the License. # -sbt/sbt clean scalastyle > scalastyle.txt -ERRORS=$(cat scalastyle.txt | grep -e "error file") +echo -e "q\n" | sbt/sbt clean scalastyle > scalastyle.txt +ERRORS=$(cat scalastyle.txt | grep -e "\") if test ! -z "$ERRORS"; then echo -e "Scalastyle checks failed at following occurrences:\n$ERRORS" exit 1 diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 730a6e7932564..9cebaf12283fc 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -6,7 +6,7 @@ title: Building Spark with Maven * This will become a table of contents (this text will be scraped). {:toc} -Building Spark using Maven Requires Maven 3 (the build process is tested with Maven 3.0.4) and Java 1.6 or newer. +Building Spark using Maven requires Maven 3.0.4 or newer and Java 1.6 or newer. ## Setting up Maven's Memory Usage ## diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index b69e3416fb322..7f75ea44e4cea 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -56,7 +56,7 @@ The recommended way to launch a compiled Spark application is through the spark- bin directory), which takes care of setting up the classpath with Spark and its dependencies, as well as provides a layer over the different cluster managers and deploy modes that Spark supports. It's usage is - spark-submit `` `` + spark-submit `` `` Where options are any of: diff --git a/docs/configuration.md b/docs/configuration.md index 1ff0150567255..57bda20edcdf1 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -122,6 +122,21 @@ Apart from these, the following properties are also available, and may be useful spark.storage.memoryFraction. + + spark.tachyonStore.baseDir + System.getProperty("java.io.tmpdir") + + Directories of the Tachyon File System that store RDDs. The Tachyon file system's URL is set by spark.tachyonStore.url. + It can also be a comma-separated list of multiple directories on Tachyon file system. + + + + spark.tachyonStore.url + tachyon://localhost:19998 + + The URL of the underlying Tachyon file system in the TachyonStore. + + spark.mesos.coarse false @@ -161,13 +176,13 @@ Apart from these, the following properties are also available, and may be useful spark.ui.acls.enable false - Whether spark web ui acls should are enabled. If enabled, this checks to see if the user has + Whether spark web ui acls should are enabled. If enabled, this checks to see if the user has access permissions to view the web ui. See spark.ui.view.acls for more details. Also note this requires the user to be known, if the user comes across as null no checks are done. Filters can be used to authenticate and set the user. - + spark.ui.view.acls Empty @@ -276,10 +291,10 @@ Apart from these, the following properties are also available, and may be useful spark.serializer.objectStreamReset 10000 - When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches - objects to prevent writing redundant data, however that stops garbage collection of those - objects. By calling 'reset' you flush that info from the serializer, and allow old - objects to be collected. To turn off this periodic reset set it to a value of <= 0. + When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches + objects to prevent writing redundant data, however that stops garbage collection of those + objects. By calling 'reset' you flush that info from the serializer, and allow old + objects to be collected. To turn off this periodic reset set it to a value of <= 0. By default it will reset the serializer every 10,000 objects. @@ -333,6 +348,32 @@ Apart from these, the following properties are also available, and may be useful receives no heartbeats. + + spark.worker.cleanup.enabled + true + + Enable periodic cleanup of worker / application directories. Note that this only affects standalone + mode, as YARN works differently. + + + + spark.worker.cleanup.interval + 1800 (30 minutes) + + Controls the interval, in seconds, at which the worker cleans up old application work dirs + on the local machine. + + + + spark.worker.cleanup.appDataTtl + 7 * 24 * 3600 (7 days) + + The number of seconds to retain application work directories on each worker. This is a Time To Live + and should depend on the amount of available disk space you have. Application logs and jars are + downloaded to each application work dir. Over time, the work dirs can quickly fill up disk space, + especially if you run jobs very frequently. + + spark.akka.frameSize 10 @@ -375,7 +416,7 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.interval 1000 - This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using failure detector can be, a sensistive failure detector can help evict rogue executors really quick. However this is usually not the case as gc pauses and network lags are expected in a real spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats between nodes leading to flooding the network with those. + This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using failure detector can be, a sensistive failure detector can help evict rogue executors really quick. However this is usually not the case as gc pauses and network lags are expected in a real spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats between nodes leading to flooding the network with those. @@ -430,7 +471,7 @@ Apart from these, the following properties are also available, and may be useful spark.broadcast.blockSize 4096 - Size of each piece of a block in kilobytes for TorrentBroadcastFactory. + Size of each piece of a block in kilobytes for TorrentBroadcastFactory. Too large a value decreases parallelism during broadcast (makes it slower); however, if it is too small, BlockManager might take a performance hit. @@ -555,7 +596,7 @@ Apart from these, the following properties are also available, and may be useful the driver. - + spark.authenticate false @@ -563,7 +604,7 @@ Apart from these, the following properties are also available, and may be useful running on Yarn. - + spark.authenticate.secret None @@ -571,12 +612,12 @@ Apart from these, the following properties are also available, and may be useful not running on Yarn and authentication is enabled. - + spark.core.connection.auth.wait.timeout 30 Number of seconds for the connection to wait for authentication to occur before timing - out and giving up. + out and giving up. diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 203d235bf9663..a5e0cc50809cf 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -38,6 +38,5 @@ depends on native Fortran routines. You may need to install the if it is not already present on your nodes. MLlib will throw a linking error if it cannot detect these libraries automatically. -To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.7 or newer -and Python 2.7. +To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.7 or newer. diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index cbe7d820b455e..888631e7025b0 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -82,15 +82,16 @@ The Python shell can be used explore data interactively and is a simple way to l >>> help(pyspark) # Show all pyspark functions {% endhighlight %} -By default, the `bin/pyspark` shell creates SparkContext that runs applications locally on a single core. -To connect to a non-local cluster, or use multiple cores, set the `MASTER` environment variable. +By default, the `bin/pyspark` shell creates SparkContext that runs applications locally on all of +your machine's logical cores. +To connect to a non-local cluster, or to specify a number of cores, set the `MASTER` environment variable. For example, to use the `bin/pyspark` shell with a [standalone Spark cluster](spark-standalone.html): {% highlight bash %} $ MASTER=spark://IP:PORT ./bin/pyspark {% endhighlight %} -Or, to use four cores on the local machine: +Or, to use exactly four cores on the local machine: {% highlight bash %} $ MASTER=local[4] ./bin/pyspark @@ -152,7 +153,7 @@ Many of the methods also contain [doctests](http://docs.python.org/2/library/doc # Libraries [MLlib](mllib-guide.html) is also available in PySpark. To use it, you'll need -[NumPy](http://www.numpy.org) version 1.7 or newer, and Python 2.7. The [MLlib guide](mllib-guide.html) contains +[NumPy](http://www.numpy.org) version 1.7 or newer. The [MLlib guide](mllib-guide.html) contains some example applications. # Where to Go from Here diff --git a/docs/quick-start.md b/docs/quick-start.md index 13df6beea16e8..60e8b1ba0eb46 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -124,7 +124,7 @@ object SimpleApp { } {% endhighlight %} -This program just counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the proogram. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes. +This program just counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the program. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes. This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` which explains that Spark is a dependency. This file also adds a repository that Spark depends on: diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 99412733d4268..a07cd2e0a32a2 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -23,7 +23,7 @@ To write a Spark application, you need to add a dependency on Spark. If you use groupId = org.apache.spark artifactId = spark-core_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION}} + version = {{site.SPARK_VERSION}} In addition, if you wish to access an HDFS cluster, you need to add a dependency on `hadoop-client` for your version of HDFS: @@ -54,7 +54,7 @@ object for more advanced configuration. The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later. -In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on four cores, use +In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on exactly four cores, use {% highlight bash %} $ MASTER=local[4] ./bin/spark-shell @@ -73,18 +73,19 @@ The master URL passed to Spark can be in one of the following formats: - - -
Master URLMeaning
local Run Spark locally with one worker thread (i.e. no parallelism at all).
local[K] Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine). +
local[K] Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine). +
local[*] Run Spark locally with as many worker threads as logical cores on your machine.
spark://HOST:PORT Connect to the given Spark standalone - cluster master. The port must be whichever one your master is configured to use, which is 7077 by default. +
spark://HOST:PORT Connect to the given Spark standalone + cluster master. The port must be whichever one your master is configured to use, which is 7077 by default.
mesos://HOST:PORT Connect to the given Mesos cluster. - The host parameter is the hostname of the Mesos master. The port must be whichever one the master is configured to use, - which is 5050 by default. +
mesos://HOST:PORT Connect to the given Mesos cluster. + The host parameter is the hostname of the Mesos master. The port must be whichever one the master is configured to use, + which is 5050 by default.
-If no master URL is specified, the spark shell defaults to "local". +If no master URL is specified, the spark shell defaults to "local[*]". For running on YARN, Spark launches an instance of the standalone deploy cluster within YARN; see [running on YARN](running-on-yarn.html) for details. @@ -265,11 +266,25 @@ A complete list of actions is available in the [RDD API doc](api/core/index.html ## RDD Persistence -One of the most important capabilities in Spark is *persisting* (or *caching*) a dataset in memory across operations. When you persist an RDD, each node stores any slices of it that it computes in memory and reuses them in other actions on that dataset (or datasets derived from it). This allows future actions to be much faster (often by more than 10x). Caching is a key tool for building iterative algorithms with Spark and for interactive use from the interpreter. - -You can mark an RDD to be persisted using the `persist()` or `cache()` methods on it. The first time it is computed in an action, it will be kept in memory on the nodes. The cache is fault-tolerant -- if any partition of an RDD is lost, it will automatically be recomputed using the transformations that originally created it. - -In addition, each RDD can be stored using a different *storage level*, allowing you, for example, to persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), or even replicate it across nodes. These levels are chosen by passing a [`org.apache.spark.storage.StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel) object to `persist()`. The `cache()` method is a shorthand for using the default storage level, which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The complete set of available storage levels is: +One of the most important capabilities in Spark is *persisting* (or *caching*) a dataset in memory +across operations. When you persist an RDD, each node stores any slices of it that it computes in +memory and reuses them in other actions on that dataset (or datasets derived from it). This allows +future actions to be much faster (often by more than 10x). Caching is a key tool for building +iterative algorithms with Spark and for interactive use from the interpreter. + +You can mark an RDD to be persisted using the `persist()` or `cache()` methods on it. The first time +it is computed in an action, it will be kept in memory on the nodes. The cache is fault-tolerant -- +if any partition of an RDD is lost, it will automatically be recomputed using the transformations +that originally created it. + +In addition, each RDD can be stored using a different *storage level*, allowing you, for example, to +persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), +or replicate it across nodes, or store the data in off-heap memory in [Tachyon](http://tachyon-project.org/). +These levels are chosen by passing a +[`org.apache.spark.storage.StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel) +object to `persist()`. The `cache()` method is a shorthand for using the default storage level, +which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The complete set of +available storage levels is: @@ -292,8 +307,16 @@ In addition, each RDD can be stored using a different *storage level*, allowing - + + + + + @@ -307,30 +330,59 @@ In addition, each RDD can be stored using a different *storage level*, allowing ### Which Storage Level to Choose? -Spark's storage levels are meant to provide different tradeoffs between memory usage and CPU efficiency. -We recommend going through the following process to select one: - -* If your RDDs fit comfortably with the default storage level (`MEMORY_ONLY`), leave them that way. This is the most - CPU-efficient option, allowing operations on the RDDs to run as fast as possible. -* If not, try using `MEMORY_ONLY_SER` and [selecting a fast serialization library](tuning.html) to make the objects - much more space-efficient, but still reasonably fast to access. -* Don't spill to disk unless the functions that computed your datasets are expensive, or they filter a large - amount of the data. Otherwise, recomputing a partition is about as fast as reading it from disk. -* Use the replicated storage levels if you want fast fault recovery (e.g. if using Spark to serve requests from a web - application). *All* the storage levels provide full fault tolerance by recomputing lost data, but the replicated ones - let you continue running tasks on the RDD without waiting to recompute a lost partition. - -If you want to define your own storage level (say, with replication factor of 3 instead of 2), then use the function factor method `apply()` of the [`StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel$) singleton object. +Spark's storage levels are meant to provide different trade-offs between memory usage and CPU +efficiency. It allows uses to choose memory, disk, or Tachyon for storing data. We recommend going +through the following process to select one: + +* If your RDDs fit comfortably with the default storage level (`MEMORY_ONLY`), leave them that way. + This is the most CPU-efficient option, allowing operations on the RDDs to run as fast as possible. + +* If not, try using `MEMORY_ONLY_SER` and [selecting a fast serialization library](tuning.html) to +make the objects much more space-efficient, but still reasonably fast to access. You can also use +`OFF_HEAP` mode to store the data off the heap in [Tachyon](http://tachyon-project.org/). This will +significantly reduce JVM GC overhead. + +* Don't spill to disk unless the functions that computed your datasets are expensive, or they filter +a large amount of the data. Otherwise, recomputing a partition is about as fast as reading it from +disk. + +* Use the replicated storage levels if you want fast fault recovery (e.g. if using Spark to serve +requests from a web application). *All* the storage levels provide full fault tolerance by +recomputing lost data, but the replicated ones let you continue running tasks on the RDD without +waiting to recompute a lost partition. + +If you want to define your own storage level (say, with replication factor of 3 instead of 2), then +use the function factor method `apply()` of the +[`StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel$) singleton object. + +Spark has a block manager inside the Executors that let you chose memory, disk, or off-heap. The +latter is for storing RDDs off-heap outside the Executor JVM on top of the memory management system +[Tachyon](http://tachyon-project.org/). This mode has the following advantages: + +* Cached data will not be lost if individual executors crash. +* Executors can have a smaller memory footprint, allowing you to run more executors on the same +machine as the bulk of the memory will be inside Tachyon. +* Reduced GC overhead since data is stored in Tachyon. # Shared Variables -Normally, when a function passed to a Spark operation (such as `map` or `reduce`) is executed on a remote cluster node, it works on separate copies of all the variables used in the function. These variables are copied to each machine, and no updates to the variables on the remote machine are propagated back to the driver program. Supporting general, read-write shared variables across tasks would be inefficient. However, Spark does provide two limited types of *shared variables* for two common usage patterns: broadcast variables and accumulators. +Normally, when a function passed to a Spark operation (such as `map` or `reduce`) is executed on a +remote cluster node, it works on separate copies of all the variables used in the function. These +variables are copied to each machine, and no updates to the variables on the remote machine are +propagated back to the driver program. Supporting general, read-write shared variables across tasks +would be inefficient. However, Spark does provide two limited types of *shared variables* for two +common usage patterns: broadcast variables and accumulators. ## Broadcast Variables -Broadcast variables allow the programmer to keep a read-only variable cached on each machine rather than shipping a copy of it with tasks. They can be used, for example, to give every node a copy of a large input dataset in an efficient manner. Spark also attempts to distribute broadcast variables using efficient broadcast algorithms to reduce communication cost. +Broadcast variables allow the programmer to keep a read-only variable cached on each machine rather +than shipping a copy of it with tasks. They can be used, for example, to give every node a copy of a +large input dataset in an efficient manner. Spark also attempts to distribute broadcast variables +using efficient broadcast algorithms to reduce communication cost. -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 interpreter session below shows this: +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 interpreter session below shows this: {% highlight scala %} scala> val broadcastVar = sc.broadcast(Array(1, 2, 3)) @@ -340,13 +392,21 @@ scala> broadcastVar.value res0: Array[Int] = Array(1, 2, 3) {% endhighlight %} -After the broadcast variable is created, it should be used instead of the value `v` in any functions run on the cluster so that `v` is not shipped to the nodes more than once. In addition, the object `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). +After the broadcast variable is created, it should be used instead of the value `v` in any functions +run on the cluster so that `v` is not shipped to the nodes more than once. In addition, the object +`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 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 MapReduce) or sums. Spark natively supports accumulators of numeric value types and standard mutable collections, and programmers can add support for new types. +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 +MapReduce) or sums. Spark natively supports accumulators of numeric value types and standard mutable +collections, and programmers can add support for new types. -An accumulator is created from an initial value `v` by calling `SparkContext.accumulator(v)`. Tasks running on the cluster can then add to it using the `+=` operator. However, they cannot read its value. Only the driver program can read the accumulator's value, using its `value` method. +An accumulator is created from an initial value `v` by calling `SparkContext.accumulator(v)`. Tasks +running on the cluster can then add to it using the `+=` operator. However, they cannot read its +value. Only the driver program can read the accumulator's value, using its `value` method. The interpreter session below shows an accumulator being used to add up the elements of an array: diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index f849716f7a48f..a59393e1424de 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -264,8 +264,8 @@ evaluated by the SQL execution engine. A full list of the functions supported c Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). However, since Hive has a large number of dependencies, it is not included in the default Spark assembly. -In order to use Hive you must first run '`SPARK_HIVE=true sbt/sbt assembly/assembly`'. This command builds a new assembly -jar that includes Hive. Note that this Hive assembly jar must also be present +In order to use Hive you must first run '`SPARK_HIVE=true sbt/sbt assembly/assembly`' (or use `-Phive` for maven). +This command builds a new assembly jar that includes Hive. Note that this Hive assembly jar must also be present on all of the worker nodes, as they will need access to the Hive serialization and deserialization libraries (SerDes) in order to acccess data stored in Hive. diff --git a/examples/pom.xml b/examples/pom.xml index a5569ff5e71f3..0b6212b5d1549 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -110,7 +110,7 @@ org.apache.hbase hbase - 0.94.6 + ${hbase.version} asm diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index e5a09ecec006f..d3babc3ed12c8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -18,8 +18,8 @@ package org.apache.spark.examples import scala.math.random + import org.apache.spark._ -import SparkContext._ /** Computes an approximation to pi */ object SparkPi { diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala new file mode 100644 index 0000000000000..53b303d658386 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala @@ -0,0 +1,80 @@ +/* + * 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 + +import java.util.Random +import scala.math.exp +import org.apache.spark.util.Vector +import org.apache.spark._ +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.scheduler.InputFormatInfo +import org.apache.spark.storage.StorageLevel + +/** + * Logistic regression based classification. + * This example uses Tachyon to persist rdds during computation. + */ +object SparkTachyonHdfsLR { + val D = 10 // Numer of dimensions + val rand = new Random(42) + + case class DataPoint(x: Vector, y: Double) + + def parsePoint(line: String): DataPoint = { + val tok = new java.util.StringTokenizer(line, " ") + var y = tok.nextToken.toDouble + var x = new Array[Double](D) + var i = 0 + while (i < D) { + x(i) = tok.nextToken.toDouble; i += 1 + } + DataPoint(new Vector(x), y) + } + + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: SparkTachyonHdfsLR ") + System.exit(1) + } + val inputPath = args(1) + val conf = SparkHadoopUtil.get.newConfiguration() + val sc = new SparkContext(args(0), "SparkTachyonHdfsLR", + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), + InputFormatInfo.computePreferredLocations( + Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) + )) + val lines = sc.textFile(inputPath) + val points = lines.map(parsePoint _).persist(StorageLevel.OFF_HEAP) + val ITERATIONS = args(2).toInt + + // Initialize w to a random value + var w = Vector(D, _ => 2 * rand.nextDouble - 1) + println("Initial w: " + w) + + for (i <- 1 to ITERATIONS) { + println("On iteration " + i) + val gradient = points.map { p => + (1 / (1 + exp(-p.y * (w dot p.x))) - 1) * p.y * p.x + }.reduce(_ + _) + w -= gradient + } + + println("Final w: " + w) + System.exit(0) + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala new file mode 100644 index 0000000000000..ce78f0876ed7c --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala @@ -0,0 +1,52 @@ +/* + * 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 + +import scala.math.random + +import org.apache.spark._ +import org.apache.spark.storage.StorageLevel + +/** + * Computes an approximation to pi + * This example uses Tachyon to persist rdds during computation. + */ +object SparkTachyonPi { + def main(args: Array[String]) { + if (args.length == 0) { + System.err.println("Usage: SparkTachyonPi []") + System.exit(1) + } + val spark = new SparkContext(args(0), "SparkTachyonPi", + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + + val slices = if (args.length > 1) args(1).toInt else 2 + val n = 100000 * slices + + val rdd = spark.parallelize(1 to n, slices) + rdd.persist(StorageLevel.OFF_HEAP) + val count = rdd.map { i => + val x = random * 2 - 1 + val y = random * 2 - 1 + if (x * x + y * y < 1) 1 else 0 + }.reduce(_ + _) + println("Pi is roughly " + 4.0 * count / n) + + spark.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala index abcc1f04d4279..62329bde84481 100644 --- a/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala @@ -33,20 +33,20 @@ object HiveFromSpark { val hiveContext = new LocalHiveContext(sc) import hiveContext._ - sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - sql("LOAD DATA LOCAL INPATH 'src/main/resources/kv1.txt' INTO TABLE src") + hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + hql("LOAD DATA LOCAL INPATH 'src/main/resources/kv1.txt' INTO TABLE src") // Queries are expressed in HiveQL println("Result of 'SELECT *': ") - sql("SELECT * FROM src").collect.foreach(println) + hql("SELECT * FROM src").collect.foreach(println) // Aggregation queries are also supported. - val count = sql("SELECT COUNT(*) FROM src").collect().head.getInt(0) + val count = hql("SELECT COUNT(*) FROM src").collect().head.getInt(0) println(s"COUNT(*): $count") // The results of SQL queries are themselves RDDs and support all normal RDD functions. The // items in the RDD are of type Row, which allows you to access each column by ordinal. - val rddFromSql = sql("SELECT key, value FROM src WHERE key < 10 ORDER BY key") + val rddFromSql = hql("SELECT key, value FROM src WHERE key < 10 ORDER BY key") println("Result of RDD.map:") val rddAsStrings = rddFromSql.map { @@ -59,6 +59,6 @@ object HiveFromSpark { // Queries can then join RDD data with data stored in Hive. println("Result of SELECT *:") - sql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) + hql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) } } diff --git a/graphx/pom.xml b/graphx/pom.xml index 5a5022916d234..b4c67ddcd8ca9 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -54,7 +54,7 @@ org.jblas jblas - 1.2.3 + ${jblas.version} org.eclipse.jetty diff --git a/mllib/pom.xml b/mllib/pom.xml index fec1cc94b2642..e7ce00efc4af6 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -58,7 +58,7 @@ org.jblas jblas - 1.2.3 + ${jblas.version} org.scalanlp diff --git a/pom.xml b/pom.xml index 7d58060cba606..c03bb35c99442 100644 --- a/pom.xml +++ b/pom.xml @@ -21,7 +21,7 @@ org.apache apache - 13 + 14 org.apache.spark spark-parent @@ -54,11 +54,11 @@ JIRA - https://spark-project.atlassian.net/browse/SPARK + https://issues.apache.org/jira/browse/SPARK - 3.0.0 + 3.0.4 @@ -123,6 +123,10 @@ 0.94.6 0.12.0 1.3.2 + 1.2.3 + 8.1.14.v20131031 + 0.3.1 + 3.0.0 64m 512m @@ -192,22 +196,22 @@ org.eclipse.jetty jetty-util - 8.1.14.v20131031 + ${jetty.version} org.eclipse.jetty jetty-security - 8.1.14.v20131031 + ${jetty.version} org.eclipse.jetty jetty-plus - 8.1.14.v20131031 + ${jetty.version} org.eclipse.jetty jetty-server - 8.1.14.v20131031 + ${jetty.version} com.google.guava @@ -273,7 +277,7 @@ com.twitter chill_${scala.binary.version} - 0.3.1 + ${chill.version} org.ow2.asm @@ -288,7 +292,7 @@ com.twitter chill-java - 0.3.1 + ${chill.version} org.ow2.asm @@ -373,7 +377,6 @@ org.apache.derby derby 10.4.2.0 - test net.liftweb @@ -392,27 +395,27 @@ com.codahale.metrics metrics-core - 3.0.0 + ${codahale.metrics.version} com.codahale.metrics metrics-jvm - 3.0.0 + ${codahale.metrics.version} com.codahale.metrics metrics-json - 3.0.0 + ${codahale.metrics.version} com.codahale.metrics metrics-ganglia - 3.0.0 + ${codahale.metrics.version} com.codahale.metrics metrics-graphite - 3.0.0 + ${codahale.metrics.version} org.scala-lang @@ -576,6 +579,12 @@ + + + org.codehaus.jackson + jackson-mapper-asl + 1.8.8 + @@ -585,7 +594,7 @@ org.apache.maven.plugins maven-enforcer-plugin - 1.1.1 + 1.3.1 enforce-versions @@ -595,7 +604,7 @@ - 3.0.0 + 3.0.4 ${java.version} @@ -608,12 +617,12 @@ org.codehaus.mojo build-helper-maven-plugin - 1.7 + 1.8 net.alchim31.maven scala-maven-plugin - 3.1.5 + 3.1.6 scala-compile-first @@ -674,7 +683,7 @@ org.apache.maven.plugins maven-surefire-plugin - 2.12.4 + 2.17 true @@ -713,7 +722,7 @@ org.apache.maven.plugins maven-shade-plugin - 2.0 + 2.2 org.apache.maven.plugins @@ -810,7 +819,6 @@ org.apache.maven.plugins maven-jar-plugin - 2.4 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c5c697e8e2427..6b8740d9f21a1 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -30,7 +30,7 @@ import scala.collection.JavaConversions._ // import com.jsuereth.pgp.sbtplugin.PgpKeys._ object SparkBuild extends Build { - val SPARK_VERSION = "1.0.0-SNAPSHOT" + val SPARK_VERSION = "1.0.0-SNAPSHOT" // Hadoop version to build against. For example, "1.0.4" for Apache releases, or // "2.0.0-mr1-cdh4.2.0" for Cloudera Hadoop. Note that these variables can be set @@ -43,6 +43,8 @@ object SparkBuild extends Build { val DEFAULT_YARN = false + val DEFAULT_HIVE = false + // HBase version; set as appropriate. val HBASE_VERSION = "0.94.6" @@ -67,15 +69,17 @@ object SparkBuild extends Build { lazy val sql = Project("sql", file("sql/core"), settings = sqlCoreSettings) dependsOn(core, catalyst) - // Since hive is its own assembly, it depends on all of the modules. - lazy val hive = Project("hive", file("sql/hive"), settings = hiveSettings) dependsOn(sql, graphx, bagel, mllib, streaming, repl) + lazy val hive = Project("hive", file("sql/hive"), settings = hiveSettings) dependsOn(sql) + + lazy val maybeHive: Seq[ClasspathDependency] = if (isHiveEnabled) Seq(hive) else Seq() + lazy val maybeHiveRef: Seq[ProjectReference] = if (isHiveEnabled) Seq(hive) else Seq() lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn(core) lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core) lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings) - .dependsOn(core, graphx, bagel, mllib, streaming, repl, sql) dependsOn(maybeYarn: _*) dependsOn(maybeGanglia: _*) + .dependsOn(core, graphx, bagel, mllib, streaming, repl, sql) dependsOn(maybeYarn: _*) dependsOn(maybeHive: _*) dependsOn(maybeGanglia: _*) lazy val assembleDeps = TaskKey[Unit]("assemble-deps", "Build assembly of dependencies and packages Spark projects") @@ -101,6 +105,11 @@ object SparkBuild extends Build { lazy val hadoopClient = if (hadoopVersion.startsWith("0.20.") || hadoopVersion == "1.0.0") "hadoop-core" else "hadoop-client" val maybeAvro = if (hadoopVersion.startsWith("0.23.") && isYarnEnabled) Seq("org.apache.avro" % "avro" % "1.7.4") else Seq() + lazy val isHiveEnabled = Properties.envOrNone("SPARK_HIVE") match { + case None => DEFAULT_HIVE + case Some(v) => v.toBoolean + } + // Include Ganglia integration if the user has enabled Ganglia // This is isolated from the normal build due to LGPL-licensed code in the library lazy val isGangliaEnabled = Properties.envOrNone("SPARK_GANGLIA_LGPL").isDefined @@ -141,13 +150,13 @@ object SparkBuild extends Build { lazy val allExternalRefs = Seq[ProjectReference](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) - .dependsOn(core, mllib, graphx, bagel, streaming, externalTwitter, hive) dependsOn(allExternal: _*) + .dependsOn(core, mllib, graphx, bagel, streaming, hive) dependsOn(allExternal: _*) // Everything except assembly, hive, tools, java8Tests and examples belong to packageProjects - lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx, catalyst, sql) ++ maybeYarnRef ++ maybeGangliaRef + lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx, catalyst, sql) ++ maybeYarnRef ++ maybeHiveRef ++ maybeGangliaRef lazy val allProjects = packageProjects ++ allExternalRefs ++ - Seq[ProjectReference](examples, tools, assemblyProj, hive) ++ maybeJava8Tests + Seq[ProjectReference](examples, tools, assemblyProj) ++ maybeJava8Tests def sharedSettings = Defaults.defaultSettings ++ MimaBuild.mimaSettings(file(sparkHome)) ++ Seq( organization := "org.apache.spark", @@ -169,6 +178,7 @@ object SparkBuild extends Build { fork := true, javaOptions in Test += "-Dspark.home=" + sparkHome, javaOptions in Test += "-Dspark.testing=1", + javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark").map { case (k,v) => s"-D$k=$v" }.toSeq, javaOptions += "-Xmx3g", // Show full stack trace and duration in test cases. @@ -185,15 +195,14 @@ object SparkBuild extends Build { concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), resolvers ++= Seq( - // HTTPS is unavailable for Maven Central "Maven Repository" at "http://repo.maven.apache.org/maven2", "Apache Repository" at "https://repository.apache.org/content/repositories/releases", "JBoss Repository" at "https://repository.jboss.org/nexus/content/repositories/releases/", "MQTT Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/", - "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/", + "Cloudera Repository" at "http://repository.cloudera.com/artifactory/cloudera-repos/", // For Sonatype publishing - //"sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", - //"sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/", + // "sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", + // "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/", // also check the local Maven repository ~/.m2 Resolver.mavenLocal ), @@ -249,10 +258,10 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "io.netty" % "netty-all" % "4.0.17.Final", - "org.eclipse.jetty" % "jetty-server" % "8.1.14.v20131031", - "org.eclipse.jetty" % "jetty-util" % "8.1.14.v20131031", - "org.eclipse.jetty" % "jetty-plus" % "8.1.14.v20131031", - "org.eclipse.jetty" % "jetty-security" % "8.1.14.v20131031", + "org.eclipse.jetty" % "jetty-server" % jettyVersion, + "org.eclipse.jetty" % "jetty-util" % jettyVersion, + "org.eclipse.jetty" % "jetty-plus" % jettyVersion, + "org.eclipse.jetty" % "jetty-security" % jettyVersion, /** Workaround for SPARK-959. Dependency used by org.eclipse.jetty. Fixed in ivy 2.3.0. */ "org.eclipse.jetty.orbit" % "javax.servlet" % "3.0.0.v201112011016" artifacts Artifact("javax.servlet", "jar", "jar"), "org.scalatest" %% "scalatest" % "1.9.1" % "test", @@ -277,16 +286,28 @@ object SparkBuild extends Build { publishLocalBoth <<= Seq(publishLocal in MavenCompile, publishLocal).dependOn ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings ++ ScalaStyleSettings + val akkaVersion = "2.2.3-shaded-protobuf" + val chillVersion = "0.3.1" + val codahaleMetricsVersion = "3.0.0" + val jblasVersion = "1.2.3" + val jettyVersion = "8.1.14.v20131031" + val hiveVersion = "0.12.0" + val parquetVersion = "1.3.2" val slf4jVersion = "1.7.5" val excludeNetty = ExclusionRule(organization = "org.jboss.netty") + val excludeEclipseJetty = ExclusionRule(organization = "org.eclipse.jetty") val excludeAsm = ExclusionRule(organization = "org.ow2.asm") val excludeOldAsm = ExclusionRule(organization = "asm") val excludeCommonsLogging = ExclusionRule(organization = "commons-logging") val excludeSLF4J = ExclusionRule(organization = "org.slf4j") val excludeScalap = ExclusionRule(organization = "org.scala-lang", artifact = "scalap") + val excludeHadoop = ExclusionRule(organization = "org.apache.hadoop") + val excludeCurator = ExclusionRule(organization = "org.apache.curator") + val excludePowermock = ExclusionRule(organization = "org.powermock") + - def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark", + def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark", version: String = "0.9.0-incubating", crossVersion: String = "2.10"): Option[sbt.ModuleID] = { val fullId = if (crossVersion.isEmpty) id else id + "_" + crossVersion Some(organization % fullId % version) // the artifact to compare binary compatibility with @@ -305,9 +326,9 @@ object SparkBuild extends Build { "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407 "com.ning" % "compress-lzf" % "1.0.0", "org.xerial.snappy" % "snappy-java" % "1.0.5", - "org.spark-project.akka" %% "akka-remote" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), - "org.spark-project.akka" %% "akka-slf4j" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), - "org.spark-project.akka" %% "akka-testkit" % "2.2.3-shaded-protobuf" % "test", + "org.spark-project.akka" %% "akka-remote" % akkaVersion excludeAll(excludeNetty), + "org.spark-project.akka" %% "akka-slf4j" % akkaVersion excludeAll(excludeNetty), + "org.spark-project.akka" %% "akka-testkit" % akkaVersion % "test", "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", @@ -317,12 +338,13 @@ object SparkBuild extends Build { "org.apache.derby" % "derby" % "10.4.2.0" % "test", "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm), "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeNetty), - "com.codahale.metrics" % "metrics-core" % "3.0.0", - "com.codahale.metrics" % "metrics-jvm" % "3.0.0", - "com.codahale.metrics" % "metrics-json" % "3.0.0", - "com.codahale.metrics" % "metrics-graphite" % "3.0.0", - "com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm), - "com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm), + "com.codahale.metrics" % "metrics-core" % codahaleMetricsVersion, + "com.codahale.metrics" % "metrics-jvm" % codahaleMetricsVersion, + "com.codahale.metrics" % "metrics-json" % codahaleMetricsVersion, + "com.codahale.metrics" % "metrics-graphite" % codahaleMetricsVersion, + "com.twitter" %% "chill" % chillVersion excludeAll(excludeAsm), + "com.twitter" % "chill-java" % chillVersion excludeAll(excludeAsm), + "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), "com.clearspring.analytics" % "stream" % "2.5.1" ), libraryDependencies ++= maybeAvro @@ -365,7 +387,7 @@ object SparkBuild extends Build { name := "spark-graphx", previousArtifact := sparkPreviousArtifact("spark-graphx"), libraryDependencies ++= Seq( - "org.jblas" % "jblas" % "1.2.3" + "org.jblas" % "jblas" % jblasVersion ) ) @@ -378,7 +400,7 @@ object SparkBuild extends Build { name := "spark-mllib", previousArtifact := sparkPreviousArtifact("spark-mllib"), libraryDependencies ++= Seq( - "org.jblas" % "jblas" % "1.2.3", + "org.jblas" % "jblas" % jblasVersion, "org.scalanlp" %% "breeze" % "0.7" ) ) @@ -398,22 +420,20 @@ object SparkBuild extends Build { def sqlCoreSettings = sharedSettings ++ Seq( name := "spark-sql", libraryDependencies ++= Seq( - "com.twitter" % "parquet-column" % "1.3.2", - "com.twitter" % "parquet-hadoop" % "1.3.2" + "com.twitter" % "parquet-column" % parquetVersion, + "com.twitter" % "parquet-hadoop" % parquetVersion ) ) // Since we don't include hive in the main assembly this project also acts as an alternative // assembly jar. - def hiveSettings = sharedSettings ++ assemblyProjSettings ++ Seq( + def hiveSettings = sharedSettings ++ Seq( name := "spark-hive", - jarName in assembly <<= version map { v => "spark-hive-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" }, - jarName in packageDependency <<= version map { v => "spark-hive-assembly-" + v + "-hadoop" + hadoopVersion + "-deps.jar" }, javaOptions += "-XX:MaxPermSize=1g", libraryDependencies ++= Seq( - "org.apache.hive" % "hive-metastore" % "0.12.0", - "org.apache.hive" % "hive-exec" % "0.12.0", - "org.apache.hive" % "hive-serde" % "0.12.0" + "org.apache.hive" % "hive-metastore" % hiveVersion, + "org.apache.hive" % "hive-exec" % hiveVersion, + "org.apache.hive" % "hive-serde" % hiveVersion ), // Multiple queries rely on the TestHive singleton. See comments there for more details. parallelExecution in Test := false, @@ -544,7 +564,7 @@ object SparkBuild extends Build { name := "spark-streaming-zeromq", previousArtifact := sparkPreviousArtifact("spark-streaming-zeromq"), libraryDependencies ++= Seq( - "org.spark-project.akka" %% "akka-zeromq" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty) + "org.spark-project.akka" %% "akka-zeromq" % akkaVersion excludeAll(excludeNetty) ) ) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index bf2454fd7e38e..d8667e84fedff 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -28,7 +28,8 @@ from pyspark.conf import SparkConf from pyspark.files import SparkFiles from pyspark.java_gateway import launch_gateway -from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer +from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ + PairDeserializer from pyspark.storagelevel import StorageLevel from pyspark import rdd from pyspark.rdd import RDD @@ -257,6 +258,45 @@ def textFile(self, name, minSplits=None): return RDD(self._jsc.textFile(name, minSplits), self, UTF8Deserializer()) + def wholeTextFiles(self, path): + """ + Read a directory of text files from HDFS, a local file system + (available on all nodes), or any Hadoop-supported file system + URI. Each file is read as a single record and returned in a + key-value pair, where the key is the path of each file, the + value is the content of each file. + + For example, if you have the following files:: + + hdfs://a-hdfs-path/part-00000 + hdfs://a-hdfs-path/part-00001 + ... + hdfs://a-hdfs-path/part-nnnnn + + Do C{rdd = sparkContext.wholeTextFiles("hdfs://a-hdfs-path")}, + then C{rdd} contains:: + + (a-hdfs-path/part-00000, its content) + (a-hdfs-path/part-00001, its content) + ... + (a-hdfs-path/part-nnnnn, its content) + + NOTE: Small files are preferred, as each file will be loaded + fully in memory. + + >>> dirPath = os.path.join(tempdir, "files") + >>> os.mkdir(dirPath) + >>> with open(os.path.join(dirPath, "1.txt"), "w") as file1: + ... file1.write("1") + >>> with open(os.path.join(dirPath, "2.txt"), "w") as file2: + ... file2.write("2") + >>> textFiles = sc.wholeTextFiles(dirPath) + >>> sorted(textFiles.collect()) + [(u'.../1.txt', u'1'), (u'.../2.txt', u'2')] + """ + return RDD(self._jsc.wholeTextFiles(path), self, + PairDeserializer(UTF8Deserializer(), UTF8Deserializer())) + def _checkpointFile(self, name, input_deserializer): jrdd = self._jsc.checkpointFile(name) return RDD(jrdd, self, input_deserializer) @@ -383,8 +423,11 @@ def _getJavaStorageLevel(self, storageLevel): raise Exception("storageLevel must be of type pyspark.StorageLevel") newStorageLevel = self._jvm.org.apache.spark.storage.StorageLevel - return newStorageLevel(storageLevel.useDisk, storageLevel.useMemory, - storageLevel.deserialized, storageLevel.replication) + return newStorageLevel(storageLevel.useDisk, + storageLevel.useMemory, + storageLevel.useOffHeap, + storageLevel.deserialized, + storageLevel.replication) def setJobGroup(self, groupId, description): """ @@ -425,7 +468,7 @@ def _test(): globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) globs['tempdir'] = tempfile.mkdtemp() atexit.register(lambda: shutil.rmtree(globs['tempdir'])) - (failure_count, test_count) = doctest.testmod(globs=globs) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: exit(-1) diff --git a/python/pyspark/mllib/__init__.py b/python/pyspark/mllib/__init__.py index b420d7a7f23ba..538ff26ce7c33 100644 --- a/python/pyspark/mllib/__init__.py +++ b/python/pyspark/mllib/__init__.py @@ -19,11 +19,7 @@ Python bindings for MLlib. """ -# MLlib currently needs Python 2.7+ and NumPy 1.7+, so complain if lower - -import sys -if sys.version_info[0:2] < (2, 7): - raise Exception("MLlib requires Python 2.7+") +# MLlib currently needs and NumPy 1.7+, so complain if lower import numpy if numpy.version.version < '1.7': diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 9943296b927dc..fb27863e07f55 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1302,11 +1302,12 @@ def getStorageLevel(self): Get the RDD's current storage level. >>> rdd1 = sc.parallelize([1,2]) >>> rdd1.getStorageLevel() - StorageLevel(False, False, False, 1) + StorageLevel(False, False, False, False, 1) """ java_storage_level = self._jrdd.getStorageLevel() storage_level = StorageLevel(java_storage_level.useDisk(), java_storage_level.useMemory(), + java_storage_level.useOffHeap(), java_storage_level.deserialized(), java_storage_level.replication()) return storage_level diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 12c63f186a2b7..b253807974a2e 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -64,6 +64,7 @@ from itertools import chain, izip, product import marshal import struct +import sys from pyspark import cloudpickle @@ -113,6 +114,11 @@ class FramedSerializer(Serializer): where C{length} is a 32-bit integer and data is C{length} bytes. """ + def __init__(self): + # On Python 2.6, we can't write bytearrays to streams, so we need to convert them + # to strings first. Check if the version number is that old. + self._only_write_strings = sys.version_info[0:2] <= (2, 6) + def dump_stream(self, iterator, stream): for obj in iterator: self._write_with_length(obj, stream) @@ -127,7 +133,10 @@ def load_stream(self, stream): def _write_with_length(self, obj, stream): serialized = self.dumps(obj) write_int(len(serialized), stream) - stream.write(serialized) + if self._only_write_strings: + stream.write(str(serialized)) + else: + stream.write(serialized) def _read_with_length(self, stream): length = read_int(stream) @@ -290,7 +299,7 @@ class MarshalSerializer(FramedSerializer): class UTF8Deserializer(Serializer): """ - Deserializes streams written by getBytes. + Deserializes streams written by String.getBytes. """ def loads(self, stream): diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 3d779faf1fa44..35e48276e3cb9 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -29,7 +29,7 @@ # this is the equivalent of ADD_JARS add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("ADD_FILES") != None else None -sc = SparkContext(os.environ.get("MASTER", "local"), "PySparkShell", pyFiles=add_files) +sc = SparkContext(os.environ.get("MASTER", "local[*]"), "PySparkShell", pyFiles=add_files) print """Welcome to ____ __ diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py index c3e3a44e8e7ab..7b6660eab231b 100644 --- a/python/pyspark/storagelevel.py +++ b/python/pyspark/storagelevel.py @@ -25,23 +25,25 @@ class StorageLevel: Also contains static constants for some commonly used storage levels, such as MEMORY_ONLY. """ - def __init__(self, useDisk, useMemory, deserialized, replication = 1): + def __init__(self, useDisk, useMemory, useOffHeap, deserialized, replication = 1): self.useDisk = useDisk self.useMemory = useMemory + self.useOffHeap = useOffHeap self.deserialized = deserialized self.replication = replication def __repr__(self): - return "StorageLevel(%s, %s, %s, %s)" % ( - self.useDisk, self.useMemory, self.deserialized, self.replication) + return "StorageLevel(%s, %s, %s, %s, %s)" % ( + self.useDisk, self.useMemory, self.useOffHeap, self.deserialized, self.replication) -StorageLevel.DISK_ONLY = StorageLevel(True, False, False) -StorageLevel.DISK_ONLY_2 = StorageLevel(True, False, False, 2) -StorageLevel.MEMORY_ONLY = StorageLevel(False, True, True) -StorageLevel.MEMORY_ONLY_2 = StorageLevel(False, True, True, 2) -StorageLevel.MEMORY_ONLY_SER = StorageLevel(False, True, False) -StorageLevel.MEMORY_ONLY_SER_2 = StorageLevel(False, True, False, 2) -StorageLevel.MEMORY_AND_DISK = StorageLevel(True, True, True) -StorageLevel.MEMORY_AND_DISK_2 = StorageLevel(True, True, True, 2) -StorageLevel.MEMORY_AND_DISK_SER = StorageLevel(True, True, False) -StorageLevel.MEMORY_AND_DISK_SER_2 = StorageLevel(True, True, False, 2) +StorageLevel.DISK_ONLY = StorageLevel(True, False, False, False) +StorageLevel.DISK_ONLY_2 = StorageLevel(True, False, False, False, 2) +StorageLevel.MEMORY_ONLY = StorageLevel(False, True, False, True) +StorageLevel.MEMORY_ONLY_2 = StorageLevel(False, True, False, True, 2) +StorageLevel.MEMORY_ONLY_SER = StorageLevel(False, True, False, False) +StorageLevel.MEMORY_ONLY_SER_2 = StorageLevel(False, True, False, False, 2) +StorageLevel.MEMORY_AND_DISK = StorageLevel(True, True, False, True) +StorageLevel.MEMORY_AND_DISK_2 = StorageLevel(True, True, False, True, 2) +StorageLevel.MEMORY_AND_DISK_SER = StorageLevel(True, True, False, False) +StorageLevel.MEMORY_AND_DISK_SER_2 = StorageLevel(True, True, False, False, 2) +StorageLevel.OFF_HEAP = StorageLevel(False, False, True, False, 1) \ No newline at end of file diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 9b1da195002c2..5a367b6bb79de 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -963,7 +963,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, case Some(m) => m case None => { val prop = System.getenv("MASTER") - if (prop != null) prop else "local" + if (prop != null) prop else "local[*]" } } master diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala b/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala index 946e71039088d..0db26c3407dff 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala @@ -7,8 +7,10 @@ package org.apache.spark.repl +import scala.reflect.io.{Path, File} import scala.tools.nsc._ import scala.tools.nsc.interpreter._ +import scala.tools.nsc.interpreter.session.JLineHistory.JLineFileHistory import scala.tools.jline.console.ConsoleReader import scala.tools.jline.console.completer._ @@ -25,7 +27,7 @@ class SparkJLineReader(_completion: => Completion) extends InteractiveReader { val consoleReader = new JLineConsoleReader() lazy val completion = _completion - lazy val history: JLineHistory = JLineHistory() + lazy val history: JLineHistory = new SparkJLineHistory private def term = consoleReader.getTerminal() def reset() = term.reset() @@ -78,3 +80,11 @@ class SparkJLineReader(_completion: => Completion) extends InteractiveReader { def readOneLine(prompt: String) = consoleReader readLine prompt def readOneKey(prompt: String) = consoleReader readOneKey prompt } + +/** Changes the default history file to not collide with the scala repl's. */ +class SparkJLineHistory extends JLineFileHistory { + import Properties.userHome + + def defaultFileName = ".spark_history" + override protected lazy val historyFile = File(Path(userHome) / defaultFileName) +} diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 0edce55a93338..9d5c6a857bb00 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -44,6 +44,10 @@ + + org.scala-lang + scala-reflect + org.apache.spark spark-core_${scala.binary.version} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 4ea80fee23e1e..5b6aea81cb7d1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -219,7 +219,7 @@ class SqlParser extends StandardTokenParsers { protected lazy val relationFactor: Parser[LogicalPlan] = ident ~ (opt(AS) ~> opt(ident)) ^^ { - case ident ~ alias => UnresolvedRelation(alias, ident) + case tableName ~ alias => UnresolvedRelation(None, tableName, alias) } | "(" ~> query ~ ")" ~ opt(AS) ~ ident ^^ { case s ~ _ ~ _ ~ a => Subquery(a, s) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 2c4bf1715b646..2d62e4cbbce01 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -70,7 +70,7 @@ package object dsl { def > (other: Expression) = GreaterThan(expr, other) def >= (other: Expression) = GreaterThanOrEqual(expr, other) def === (other: Expression) = Equals(expr, other) - def != (other: Expression) = Not(Equals(expr, other)) + def !== (other: Expression) = Not(Equals(expr, other)) def like(other: Expression) = Like(expr, other) def rlike(other: Expression) = RLike(expr, other) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index f70e80b7f27f2..4ebf6c4584b94 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -45,14 +45,20 @@ case class BoundReference(ordinal: Int, baseReference: Attribute) override def toString = s"$baseReference:$ordinal" - override def apply(input: Row): Any = input(ordinal) + override def eval(input: Row): Any = input(ordinal) } +/** + * Used to denote operators that do their own binding of attributes internally. + */ +trait NoBind { self: trees.TreeNode[_] => } + class BindReferences[TreeNode <: QueryPlan[TreeNode]] extends Rule[TreeNode] { import BindReferences._ def apply(plan: TreeNode): TreeNode = { plan.transform { + case n: NoBind => n.asInstanceOf[TreeNode] case leafNode if leafNode.children.isEmpty => leafNode case unaryNode if unaryNode.children.size == 1 => unaryNode.transformExpressions { case e => bindReference(e, unaryNode.children.head.output) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 941b53fe70d23..89226999ca005 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -185,8 +185,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case DoubleType => castToDouble } - override def apply(input: Row): Any = { - val evaluated = child.apply(input) + override def eval(input: Row): Any = { + val evaluated = child.eval(input) if (evaluated == null) { null } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index a3d19525503ba..f190bd0cca375 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.types.{DataType, FractionalType, IntegralType, NumericType, NativeType} @@ -50,7 +50,7 @@ abstract class Expression extends TreeNode[Expression] { def references: Set[Attribute] /** Returns the result of evaluating this expression on a given input Row */ - def apply(input: Row = null): EvaluatedType = + def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") /** @@ -73,7 +73,7 @@ abstract class Expression extends TreeNode[Expression] { */ @inline def n1(e: Expression, i: Row, f: ((Numeric[Any], Any) => Any)): Any = { - val evalE = e.apply(i) + val evalE = e.eval(i) if (evalE == null) { null } else { @@ -102,11 +102,11 @@ abstract class Expression extends TreeNode[Expression] { throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") } - val evalE1 = e1.apply(i) + val evalE1 = e1.eval(i) if(evalE1 == null) { null } else { - val evalE2 = e2.apply(i) + val evalE2 = e2.eval(i) if (evalE2 == null) { null } else { @@ -135,11 +135,11 @@ abstract class Expression extends TreeNode[Expression] { throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") } - val evalE1 = e1.apply(i: Row) + val evalE1 = e1.eval(i: Row) if(evalE1 == null) { null } else { - val evalE2 = e2.apply(i: Row) + val evalE2 = e2.eval(i: Row) if (evalE2 == null) { null } else { @@ -168,11 +168,11 @@ abstract class Expression extends TreeNode[Expression] { throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") } - val evalE1 = e1.apply(i) + val evalE1 = e1.eval(i) if(evalE1 == null) { null } else { - val evalE2 = e2.apply(i) + val evalE2 = e2.eval(i) if (evalE2 == null) { null } else { @@ -205,11 +205,11 @@ abstract class Expression extends TreeNode[Expression] { throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") } - val evalE1 = e1.apply(i) + val evalE1 = e1.eval(i) if(evalE1 == null) { null } else { - val evalE2 = e2.apply(i) + val evalE2 = e2.eval(i) if (evalE2 == null) { null } else { @@ -231,7 +231,7 @@ abstract class BinaryExpression extends Expression with trees.BinaryNode[Express override def foldable = left.foldable && right.foldable - def references = left.references ++ right.references + override def references = left.references ++ right.references override def toString = s"($left $symbol $right)" } @@ -243,5 +243,5 @@ abstract class LeafExpression extends Expression with trees.LeafNode[Expression] abstract class UnaryExpression extends Expression with trees.UnaryNode[Expression] { self: Product => - def references = child.references + override def references = child.references } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 38542d3fc7290..c9b7cea6a3e5f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -27,11 +27,12 @@ class Projection(expressions: Seq[Expression]) extends (Row => Row) { this(expressions.map(BindReferences.bindReference(_, inputSchema))) protected val exprArray = expressions.toArray + def apply(input: Row): Row = { - val outputArray = new Array[Any](exprArray.size) + val outputArray = new Array[Any](exprArray.length) var i = 0 - while (i < exprArray.size) { - outputArray(i) = exprArray(i).apply(input) + while (i < exprArray.length) { + outputArray(i) = exprArray(i).eval(input) i += 1 } new GenericRow(outputArray) @@ -57,8 +58,8 @@ case class MutableProjection(expressions: Seq[Expression]) extends (Row => Row) def apply(input: Row): Row = { var i = 0 - while (i < exprArray.size) { - mutableRow(i) = exprArray(i).apply(input) + while (i < exprArray.length) { + mutableRow(i) = exprArray(i).eval(input) i += 1 } mutableRow diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 6f939e6c41f6b..0f06ea088e1a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -19,6 +19,21 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types.NativeType +object Row { + /** + * This method can be used to extract fields from a [[Row]] object in a pattern match. Example: + * {{{ + * import org.apache.spark.sql._ + * + * val pairs = sql("SELECT key, value FROM src").rdd.map { + * case Row(key: Int, value: String) => + * key -> value + * } + * }}} + */ + def unapplySeq(row: Row): Some[Seq[Any]] = Some(row) +} + /** * Represents one row of output from a relational operator. Allows both generic access by ordinal, * which will incur boxing overhead for primitives, as well as native primitive access. @@ -197,8 +212,8 @@ class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { var i = 0 while (i < ordering.size) { val order = ordering(i) - val left = order.child.apply(a) - val right = order.child.apply(b) + val left = order.child.eval(a) + val right = order.child.eval(b) if (left == null && right == null) { // Both null, continue looking. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index f53d8504b083f..5e089f7618e0a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -27,13 +27,13 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi def references = children.flatMap(_.references).toSet def nullable = true - override def apply(input: Row): Any = { + override def eval(input: Row): Any = { children.size match { - case 1 => function.asInstanceOf[(Any) => Any](children(0).apply(input)) + case 1 => function.asInstanceOf[(Any) => Any](children(0).eval(input)) case 2 => function.asInstanceOf[(Any, Any) => Any]( - children(0).apply(input), - children(1).apply(input)) + children(0).eval(input), + children(1).eval(input)) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala index 9828d0b9bd8b2..e787c59e75723 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala @@ -30,7 +30,7 @@ case class WrapDynamic(children: Seq[Attribute]) extends Expression { def references = children.toSet def dataType = DynamicType - override def apply(input: Row): DynamicRow = input match { + override def eval(input: Row): DynamicRow = input match { // Avoid copy for generic rows. case g: GenericRow => new DynamicRow(children, g.values) case otherRowType => new DynamicRow(children, otherRowType.toArray) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 7303b155cae3d..5edcea14278c7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -27,7 +27,7 @@ abstract class AggregateExpression extends Expression { * Creates a new instance that can be used to compute this aggregate expression for a group * of input rows/ */ - def newInstance: AggregateFunction + def newInstance(): AggregateFunction } /** @@ -43,7 +43,7 @@ case class SplitEvaluation( partialEvaluations: Seq[NamedExpression]) /** - * An [[AggregateExpression]] that can be partially computed without seeing all relevent tuples. + * An [[AggregateExpression]] that can be partially computed without seeing all relevant tuples. * These partial evaluations can then be combined to compute the actual answer. */ abstract class PartialAggregate extends AggregateExpression { @@ -63,48 +63,48 @@ abstract class AggregateFunction extends AggregateExpression with Serializable with trees.LeafNode[Expression] { self: Product => - type EvaluatedType = Any + override type EvaluatedType = Any /** Base should return the generic aggregate expression that this function is computing */ val base: AggregateExpression - def references = base.references - def nullable = base.nullable - def dataType = base.dataType + override def references = base.references + override def nullable = base.nullable + override def dataType = base.dataType def update(input: Row): Unit - override def apply(input: Row): Any + override def eval(input: Row): Any // Do we really need this? - def newInstance = makeCopy(productIterator.map { case a: AnyRef => a }.toArray) + override def newInstance() = makeCopy(productIterator.map { case a: AnyRef => a }.toArray) } case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - def references = child.references - def nullable = false - def dataType = IntegerType + override def references = child.references + override def nullable = false + override def dataType = IntegerType override def toString = s"COUNT($child)" - def asPartial: SplitEvaluation = { + override def asPartial: SplitEvaluation = { val partialCount = Alias(Count(child), "PartialCount")() SplitEvaluation(Sum(partialCount.toAttribute), partialCount :: Nil) } - override def newInstance = new CountFunction(child, this) + override def newInstance()= new CountFunction(child, this) } case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpression { - def children = expressions - def references = expressions.flatMap(_.references).toSet - def nullable = false - def dataType = IntegerType + override def children = expressions + override def references = expressions.flatMap(_.references).toSet + override def nullable = false + override def dataType = IntegerType override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")}})" - override def newInstance = new CountDistinctFunction(expressions, this) + override def newInstance()= new CountDistinctFunction(expressions, this) } case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - def references = child.references - def nullable = false - def dataType = DoubleType + override def references = child.references + override def nullable = false + override def dataType = DoubleType override def toString = s"AVG($child)" override def asPartial: SplitEvaluation = { @@ -118,13 +118,13 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN partialCount :: partialSum :: Nil) } - override def newInstance = new AverageFunction(child, this) + override def newInstance()= new AverageFunction(child, this) } case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - def references = child.references - def nullable = false - def dataType = child.dataType + override def references = child.references + override def nullable = false + override def dataType = child.dataType override def toString = s"SUM($child)" override def asPartial: SplitEvaluation = { @@ -134,24 +134,24 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ partialSum :: Nil) } - override def newInstance = new SumFunction(child, this) + override def newInstance()= new SumFunction(child, this) } case class SumDistinct(child: Expression) extends AggregateExpression with trees.UnaryNode[Expression] { - def references = child.references - def nullable = false - def dataType = child.dataType + override def references = child.references + override def nullable = false + override def dataType = child.dataType override def toString = s"SUM(DISTINCT $child)" - override def newInstance = new SumDistinctFunction(child, this) + override def newInstance()= new SumDistinctFunction(child, this) } case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - def references = child.references - def nullable = child.nullable - def dataType = child.dataType + override def references = child.references + override def nullable = child.nullable + override def dataType = child.dataType override def toString = s"FIRST($child)" override def asPartial: SplitEvaluation = { @@ -160,7 +160,7 @@ case class First(child: Expression) extends PartialAggregate with trees.UnaryNod First(partialFirst.toAttribute), partialFirst :: Nil) } - override def newInstance = new FirstFunction(child, this) + override def newInstance()= new FirstFunction(child, this) } case class AverageFunction(expr: Expression, base: AggregateExpression) @@ -169,17 +169,15 @@ case class AverageFunction(expr: Expression, base: AggregateExpression) def this() = this(null, null) // Required for serialization. private var count: Long = _ - private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).apply(EmptyRow)) + private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).eval(EmptyRow)) private val sumAsDouble = Cast(sum, DoubleType) - - private val addFunction = Add(sum, expr) - override def apply(input: Row): Any = - sumAsDouble.apply(EmptyRow).asInstanceOf[Double] / count.toDouble + override def eval(input: Row): Any = + sumAsDouble.eval(EmptyRow).asInstanceOf[Double] / count.toDouble - def update(input: Row): Unit = { + override def update(input: Row): Unit = { count += 1 sum.update(addFunction, input) } @@ -190,28 +188,28 @@ case class CountFunction(expr: Expression, base: AggregateExpression) extends Ag var count: Int = _ - def update(input: Row): Unit = { - val evaluatedExpr = expr.map(_.apply(input)) + override def update(input: Row): Unit = { + val evaluatedExpr = expr.map(_.eval(input)) if (evaluatedExpr.map(_ != null).reduceLeft(_ || _)) { count += 1 } } - override def apply(input: Row): Any = count + override def eval(input: Row): Any = count } case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. - private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).apply(null)) + private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).eval(null)) private val addFunction = Add(sum, expr) - def update(input: Row): Unit = { + override def update(input: Row): Unit = { sum.update(addFunction, input) } - override def apply(input: Row): Any = sum.apply(null) + override def eval(input: Row): Any = sum.eval(null) } case class SumDistinctFunction(expr: Expression, base: AggregateExpression) @@ -219,16 +217,16 @@ case class SumDistinctFunction(expr: Expression, base: AggregateExpression) def this() = this(null, null) // Required for serialization. - val seen = new scala.collection.mutable.HashSet[Any]() + private val seen = new scala.collection.mutable.HashSet[Any]() - def update(input: Row): Unit = { - val evaluatedExpr = expr.apply(input) + override def update(input: Row): Unit = { + val evaluatedExpr = expr.eval(input) if (evaluatedExpr != null) { seen += evaluatedExpr } } - override def apply(input: Row): Any = + override def eval(input: Row): Any = seen.reduceLeft(base.dataType.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].plus) } @@ -239,14 +237,14 @@ case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpressio val seen = new scala.collection.mutable.HashSet[Any]() - def update(input: Row): Unit = { - val evaluatedExpr = expr.map(_.apply(input)) + override def update(input: Row): Unit = { + val evaluatedExpr = expr.map(_.eval(input)) if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) { seen += evaluatedExpr } } - override def apply(input: Row): Any = seen.size + override def eval(input: Row): Any = seen.size } case class FirstFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -254,11 +252,11 @@ case class FirstFunction(expr: Expression, base: AggregateExpression) extends Ag var result: Any = null - def update(input: Row): Unit = { + override def update(input: Row): Unit = { if (result == null) { - result = expr.apply(input) + result = expr.eval(input) } } - override def apply(input: Row): Any = result + override def eval(input: Row): Any = result } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index fba056e7c07e3..c79c1847cedf5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -28,7 +28,7 @@ case class UnaryMinus(child: Expression) extends UnaryExpression { def nullable = child.nullable override def toString = s"-$child" - override def apply(input: Row): Any = { + override def eval(input: Row): Any = { n1(child, input, _.negate(_)) } } @@ -55,25 +55,25 @@ abstract class BinaryArithmetic extends BinaryExpression { case class Add(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "+" - override def apply(input: Row): Any = n2(input, left, right, _.plus(_, _)) + override def eval(input: Row): Any = n2(input, left, right, _.plus(_, _)) } case class Subtract(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "-" - override def apply(input: Row): Any = n2(input, left, right, _.minus(_, _)) + override def eval(input: Row): Any = n2(input, left, right, _.minus(_, _)) } case class Multiply(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "*" - override def apply(input: Row): Any = n2(input, left, right, _.times(_, _)) + override def eval(input: Row): Any = n2(input, left, right, _.times(_, _)) } case class Divide(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "/" - override def apply(input: Row): Any = dataType match { + override def eval(input: Row): Any = dataType match { case _: FractionalType => f2(input, left, right, _.div(_, _)) case _: IntegralType => i2(input, left , right, _.quot(_, _)) } @@ -83,5 +83,5 @@ case class Divide(left: Expression, right: Expression) extends BinaryArithmetic case class Remainder(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "%" - override def apply(input: Row): Any = i2(input, left, right, _.rem(_, _)) + override def eval(input: Row): Any = i2(input, left, right, _.rem(_, _)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index ab96618d73df7..c947155cb701c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -39,10 +39,10 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { override def toString = s"$child[$ordinal]" - override def apply(input: Row): Any = { + override def eval(input: Row): Any = { if (child.dataType.isInstanceOf[ArrayType]) { - val baseValue = child.apply(input).asInstanceOf[Seq[_]] - val o = ordinal.apply(input).asInstanceOf[Int] + val baseValue = child.eval(input).asInstanceOf[Seq[_]] + val o = ordinal.eval(input).asInstanceOf[Int] if (baseValue == null) { null } else if (o >= baseValue.size || o < 0) { @@ -51,8 +51,8 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { baseValue(o) } } else { - val baseValue = child.apply(input).asInstanceOf[Map[Any, _]] - val key = ordinal.apply(input) + val baseValue = child.eval(input).asInstanceOf[Map[Any, _]] + val key = ordinal.eval(input) if (baseValue == null) { null } else { @@ -85,8 +85,8 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio override lazy val resolved = childrenResolved && child.dataType.isInstanceOf[StructType] - override def apply(input: Row): Any = { - val baseValue = child.apply(input).asInstanceOf[Row] + override def eval(input: Row): Any = { + val baseValue = child.eval(input).asInstanceOf[Row] if (baseValue == null) null else baseValue(ordinal) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index e9b491b10a5f2..dd78614754e12 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -35,17 +35,17 @@ import org.apache.spark.sql.catalyst.types._ * requested. The attributes produced by this function will be automatically copied anytime rules * result in changes to the Generator or its children. */ -abstract class Generator extends Expression with (Row => TraversableOnce[Row]) { +abstract class Generator extends Expression { self: Product => - type EvaluatedType = TraversableOnce[Row] + override type EvaluatedType = TraversableOnce[Row] - lazy val dataType = + override lazy val dataType = ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable)))) - def nullable = false + override def nullable = false - def references = children.flatMap(_.references).toSet + override def references = children.flatMap(_.references).toSet /** * Should be overridden by specific generators. Called only once for each instance to ensure @@ -63,7 +63,7 @@ abstract class Generator extends Expression with (Row => TraversableOnce[Row]) { } /** Should be implemented by child classes to perform specific Generators. */ - def apply(input: Row): TraversableOnce[Row] + override def eval(input: Row): TraversableOnce[Row] /** Overridden `makeCopy` also copies the attributes that are produced by this generator. */ override def makeCopy(newArgs: Array[AnyRef]): this.type = { @@ -83,7 +83,7 @@ case class Explode(attributeNames: Seq[String], child: Expression) child.resolved && (child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType]) - lazy val elementTypes = child.dataType match { + private lazy val elementTypes = child.dataType match { case ArrayType(et) => et :: Nil case MapType(kt,vt) => kt :: vt :: Nil } @@ -100,13 +100,13 @@ case class Explode(attributeNames: Seq[String], child: Expression) } } - override def apply(input: Row): TraversableOnce[Row] = { + override def eval(input: Row): TraversableOnce[Row] = { child.dataType match { case ArrayType(_) => - val inputArray = child.apply(input).asInstanceOf[Seq[Any]] + val inputArray = child.eval(input).asInstanceOf[Seq[Any]] if (inputArray == null) Nil else inputArray.map(v => new GenericRow(Array(v))) case MapType(_, _) => - val inputMap = child.apply(input).asInstanceOf[Map[Any,Any]] + val inputMap = child.eval(input).asInstanceOf[Map[Any,Any]] if (inputMap == null) Nil else inputMap.map { case (k,v) => new GenericRow(Array(k,v)) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index d879b2b5e8ba1..e15e16d633365 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -57,7 +57,7 @@ case class Literal(value: Any, dataType: DataType) extends LeafExpression { override def toString = if (value != null) value.toString else "null" type EvaluatedType = Any - override def apply(input: Row):Any = value + override def eval(input: Row):Any = value } // TODO: Specialize @@ -69,8 +69,8 @@ case class MutableLiteral(var value: Any, nullable: Boolean = true) extends Leaf def references = Set.empty def update(expression: Expression, input: Row) = { - value = expression.apply(input) + value = expression.eval(input) } - override def apply(input: Row) = value + override def eval(input: Row) = value } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 69c8bed309c18..eb4bc8e755284 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -79,7 +79,7 @@ case class Alias(child: Expression, name: String) type EvaluatedType = Any - override def apply(input: Row) = child.apply(input) + override def eval(input: Row) = child.eval(input) def dataType = child.dataType def nullable = child.nullable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index 5a47768dcb4a1..ce6d99c911ab3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -41,11 +41,11 @@ case class Coalesce(children: Seq[Expression]) extends Expression { throw new UnresolvedException(this, "Coalesce cannot have children of different types.") } - override def apply(input: Row): Any = { + override def eval(input: Row): Any = { var i = 0 var result: Any = null while(i < children.size && result == null) { - result = children(i).apply(input) + result = children(i).eval(input) i += 1 } result @@ -57,8 +57,8 @@ case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expr override def foldable = child.foldable def nullable = false - override def apply(input: Row): Any = { - child.apply(input) == null + override def eval(input: Row): Any = { + child.eval(input) == null } } @@ -68,7 +68,7 @@ case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[E def nullable = false override def toString = s"IS NOT NULL $child" - override def apply(input: Row): Any = { - child.apply(input) != null + override def eval(input: Row): Any = { + child.eval(input) != null } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index b74809e5ca67d..da5b2cf5b0362 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.types.{BooleanType, StringType, TimestampTy object InterpretedPredicate { def apply(expression: Expression): (Row => Boolean) = { - (r: Row) => expression.apply(r).asInstanceOf[Boolean] + (r: Row) => expression.eval(r).asInstanceOf[Boolean] } } @@ -54,8 +54,8 @@ case class Not(child: Expression) extends Predicate with trees.UnaryNode[Express def nullable = child.nullable override def toString = s"NOT $child" - override def apply(input: Row): Any = { - child.apply(input) match { + override def eval(input: Row): Any = { + child.eval(input) match { case null => null case b: Boolean => !b } @@ -71,18 +71,18 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate { def nullable = true // TODO: Figure out correct nullability semantics of IN. override def toString = s"$value IN ${list.mkString("(", ",", ")")}" - override def apply(input: Row): Any = { - val evaluatedValue = value.apply(input) - list.exists(e => e.apply(input) == evaluatedValue) + override def eval(input: Row): Any = { + val evaluatedValue = value.eval(input) + list.exists(e => e.eval(input) == evaluatedValue) } } case class And(left: Expression, right: Expression) extends BinaryPredicate { def symbol = "&&" - override def apply(input: Row): Any = { - val l = left.apply(input) - val r = right.apply(input) + override def eval(input: Row): Any = { + val l = left.eval(input) + val r = right.eval(input) if (l == false || r == false) { false } else if (l == null || r == null ) { @@ -96,9 +96,9 @@ case class And(left: Expression, right: Expression) extends BinaryPredicate { case class Or(left: Expression, right: Expression) extends BinaryPredicate { def symbol = "||" - override def apply(input: Row): Any = { - val l = left.apply(input) - val r = right.apply(input) + override def eval(input: Row): Any = { + val l = left.eval(input) + val r = right.eval(input) if (l == true || r == true) { true } else if (l == null || r == null) { @@ -115,31 +115,31 @@ abstract class BinaryComparison extends BinaryPredicate { case class Equals(left: Expression, right: Expression) extends BinaryComparison { def symbol = "=" - override def apply(input: Row): Any = { - val l = left.apply(input) - val r = right.apply(input) + override def eval(input: Row): Any = { + val l = left.eval(input) + val r = right.eval(input) if (l == null || r == null) null else l == r } } case class LessThan(left: Expression, right: Expression) extends BinaryComparison { def symbol = "<" - override def apply(input: Row): Any = c2(input, left, right, _.lt(_, _)) + override def eval(input: Row): Any = c2(input, left, right, _.lt(_, _)) } case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { def symbol = "<=" - override def apply(input: Row): Any = c2(input, left, right, _.lteq(_, _)) + override def eval(input: Row): Any = c2(input, left, right, _.lteq(_, _)) } case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison { def symbol = ">" - override def apply(input: Row): Any = c2(input, left, right, _.gt(_, _)) + override def eval(input: Row): Any = c2(input, left, right, _.gt(_, _)) } case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { def symbol = ">=" - override def apply(input: Row): Any = c2(input, left, right, _.gteq(_, _)) + override def eval(input: Row): Any = c2(input, left, right, _.gteq(_, _)) } case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) @@ -159,11 +159,11 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi } type EvaluatedType = Any - override def apply(input: Row): Any = { - if (predicate(input).asInstanceOf[Boolean]) { - trueValue.apply(input) + override def eval(input: Row): Any = { + if (predicate.eval(input).asInstanceOf[Boolean]) { + trueValue.eval(input) } else { - falseValue.apply(input) + falseValue.eval(input) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index 42b7a9b125b7a..a27c71db1b999 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -22,8 +22,6 @@ import java.util.regex.Pattern import org.apache.spark.sql.catalyst.types.DataType import org.apache.spark.sql.catalyst.types.StringType import org.apache.spark.sql.catalyst.types.BooleanType -import org.apache.spark.sql.catalyst.trees.TreeNode -import org.apache.spark.sql.catalyst.errors.`package`.TreeNodeException trait StringRegexExpression { @@ -52,12 +50,12 @@ trait StringRegexExpression { protected def pattern(str: String) = if(cache == null) compile(str) else cache - override def apply(input: Row): Any = { - val l = left.apply(input) - if(l == null) { + override def eval(input: Row): Any = { + val l = left.eval(input) + if (l == null) { null } else { - val r = right.apply(input) + val r = right.eval(input) if(r == null) { null } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 3dd6818029bcf..37b23ba58289c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -45,7 +45,7 @@ object ConstantFolding extends Rule[LogicalPlan] { case q: LogicalPlan => q transformExpressionsDown { // Skip redundant folding of literals. case l: Literal => l - case e if e.foldable => Literal(e.apply(null), e.dataType) + case e if e.foldable => Literal(e.eval(null), e.dataType) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index b39c2b32cc42c..cfc0b0c3a8d98 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -162,6 +162,7 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode { a.nullable)( a.exprId, a.qualifiers) + case other => other } def references = Set.empty diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala index 9ec31689b5098..4589129cd1c90 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala @@ -32,18 +32,5 @@ package object sql { type Row = catalyst.expressions.Row - object Row { - /** - * This method can be used to extract fields from a [[Row]] object in a pattern match. Example: - * {{{ - * import org.apache.spark.sql._ - * - * val pairs = sql("SELECT key, value FROM src").rdd.map { - * case Row(key: Int, value: String) => - * key -> value - * } - * }}} - */ - def unapplySeq(row: Row): Some[Seq[Any]] = Some(row) - } + val Row = catalyst.expressions.Row } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 43876033d327b..92987405aa313 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ class ExpressionEvaluationSuite extends FunSuite { test("literals") { - assert((Literal(1) + Literal(1)).apply(null) === 2) + assert((Literal(1) + Literal(1)).eval(null) === 2) } /** @@ -62,7 +62,7 @@ class ExpressionEvaluationSuite extends FunSuite { notTrueTable.foreach { case (v, answer) => val expr = Not(Literal(v, BooleanType)) - val result = expr.apply(null) + val result = expr.eval(null) if (result != answer) fail(s"$expr should not evaluate to $result, expected: $answer") } } @@ -105,7 +105,7 @@ class ExpressionEvaluationSuite extends FunSuite { truthTable.foreach { case (l,r,answer) => val expr = op(Literal(l, BooleanType), Literal(r, BooleanType)) - val result = expr.apply(null) + val result = expr.eval(null) if (result != answer) fail(s"$expr should not evaluate to $result, expected: $answer") } @@ -113,7 +113,7 @@ class ExpressionEvaluationSuite extends FunSuite { } def evaluate(expression: Expression, inputRow: Row = EmptyRow): Any = { - expression.apply(inputRow) + expression.eval(inputRow) } def checkEvaluation(expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala b/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala deleted file mode 100644 index f1230e7526ab1..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.rdd - -import scala.language.implicitConversions - -import scala.reflect._ -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.{Aggregator, InterruptibleIterator, Logging} -import org.apache.spark.util.collection.AppendOnlyMap - -/* Implicit conversions */ -import org.apache.spark.SparkContext._ - -/** - * Extra functions on RDDs that perform only local operations. These can be used when data has - * already been partitioned correctly. - */ -private[spark] class PartitionLocalRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) - extends Logging - with Serializable { - - /** - * Cogroup corresponding partitions of `this` and `other`. These two RDDs should have - * the same number of partitions. Partitions of these two RDDs are cogrouped - * according to the indexes of partitions. If we have two RDDs and - * each of them has n partitions, we will cogroup the partition i from `this` - * with the partition i from `other`. - * This function will not introduce a shuffling operation. - */ - def cogroupLocally[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { - val cg = self.zipPartitions(other)((iter1:Iterator[(K, V)], iter2:Iterator[(K, W)]) => { - val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]] - - val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => { - if (hadVal) oldVal else Array.fill(2)(new ArrayBuffer[Any]) - } - - val getSeq = (k: K) => { - map.changeValue(k, update) - } - - iter1.foreach { kv => getSeq(kv._1)(0) += kv._2 } - iter2.foreach { kv => getSeq(kv._1)(1) += kv._2 } - - map.iterator - }).mapValues { case Seq(vs, ws) => (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])} - - cg - } - - /** - * Group the values for each key within a partition of the RDD into a single sequence. - * This function will not introduce a shuffling operation. - */ - def groupByKeyLocally(): RDD[(K, Seq[V])] = { - def createCombiner(v: V) = ArrayBuffer(v) - def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v - val aggregator = new Aggregator[K, V, ArrayBuffer[V]](createCombiner, mergeValue, _ ++ _) - val bufs = self.mapPartitionsWithContext((context, iter) => { - new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context)) - }, preservesPartitioning = true) - bufs.asInstanceOf[RDD[(K, Seq[V])]] - } - - /** - * Join corresponding partitions of `this` and `other`. - * If we have two RDDs and each of them has n partitions, - * we will join the partition i from `this` with the partition i from `other`. - * This function will not introduce a shuffling operation. - */ - def joinLocally[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = { - cogroupLocally(other).flatMapValues { - case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (v, w) - } - } -} - -private[spark] object PartitionLocalRDDFunctions { - implicit def rddToPartitionLocalRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = - new PartitionLocalRDDFunctions(rdd) -} - - diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 36059c6630aa4..3193787680d16 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -224,6 +224,8 @@ class SQLContext(@transient val sparkContext: SparkContext) protected def stringOrError[A](f: => A): String = try f.toString catch { case e: Throwable => e.toString } + def simpleString: String = stringOrError(executedPlan) + override def toString: String = s"""== Logical Plan == |${stringOrError(analyzed)} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index a62cb8aa1321f..fc95781448569 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -148,17 +148,17 @@ class SchemaRDD( * * @param otherPlan the [[SchemaRDD]] that should be joined with this one. * @param joinType One of `Inner`, `LeftOuter`, `RightOuter`, or `FullOuter`. Defaults to `Inner.` - * @param condition An optional condition for the join operation. This is equivilent to the `ON` - * clause in standard SQL. In the case of `Inner` joins, specifying a - * `condition` is equivilent to adding `where` clauses after the `join`. + * @param on An optional condition for the join operation. This is equivilent to the `ON` + * clause in standard SQL. In the case of `Inner` joins, specifying a + * `condition` is equivilent to adding `where` clauses after the `join`. * * @group Query */ def join( otherPlan: SchemaRDD, joinType: JoinType = Inner, - condition: Option[Expression] = None): SchemaRDD = - new SchemaRDD(sqlContext, Join(logicalPlan, otherPlan.logicalPlan, joinType, condition)) + on: Option[Expression] = None): SchemaRDD = + new SchemaRDD(sqlContext, Join(logicalPlan, otherPlan.logicalPlan, joinType, on)) /** * Sorts the results by the given expressions. @@ -195,14 +195,14 @@ class SchemaRDD( * with the same name, for example, when peforming self-joins. * * {{{ - * val x = schemaRDD.where('a === 1).subquery('x) - * val y = schemaRDD.where('a === 2).subquery('y) + * val x = schemaRDD.where('a === 1).as('x) + * val y = schemaRDD.where('a === 2).as('y) * x.join(y).where("x.a".attr === "y.a".attr), * }}} * * @group Query */ - def subquery(alias: Symbol) = + def as(alias: Symbol) = new SchemaRDD(sqlContext, Subquery(alias.name, logicalPlan)) /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 840803a52c1cf..3dd9897c0d3b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -41,8 +41,7 @@ trait SchemaRDDLike { override def toString = s"""${super.toString} |== Query Plan == - |${queryExecution.executedPlan}""".stripMargin.trim - + |${queryExecution.simpleString}""".stripMargin.trim /** * Saves the contents of this `SchemaRDD` as a parquet file, preserving the schema. Files that diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 869673b1fe978..450c142c0baa4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -76,7 +76,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una */ object AddExchange extends Rule[SparkPlan] { // TODO: Determine the number of partitions. - val numPartitions = 8 + val numPartitions = 150 def apply(plan: SparkPlan): SparkPlan = plan.transformUp { case operator: SparkPlan => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala index e902e6ced521d..cff4887936ae1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala @@ -36,10 +36,10 @@ case class Generate( child: SparkPlan) extends UnaryNode { - def output = + override def output = if (join) child.output ++ generator.output else generator.output - def execute() = { + override def execute() = { if (join) { child.execute().mapPartitions { iter => val nullValues = Seq.fill(generator.output.size)(Literal(null)) @@ -52,7 +52,7 @@ case class Generate( val joinedRow = new JoinedRow iter.flatMap {row => - val outputRows = generator(row) + val outputRows = generator.eval(row) if (outer && outputRows.isEmpty) { outerProjection(row) :: Nil } else { @@ -61,7 +61,7 @@ case class Generate( } } } else { - child.execute().mapPartitions(iter => iter.flatMap(generator)) + child.execute().mapPartitions(iter => iter.flatMap(row => generator.eval(row))) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index acb1ee83a72f6..daa423cb8ea1a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.sql.catalyst.plans.{QueryPlan, logical} import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.columnar.InMemoryColumnarTableScan abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { self: Product => @@ -69,6 +70,8 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan) SparkLogicalPlan( alreadyPlanned match { case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) + case InMemoryColumnarTableScan(output, child) => + InMemoryColumnarTableScan(output.map(_.newInstance), child) case _ => sys.error("Multiple instance of the same relation detected.") }).asInstanceOf[this.type] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala index 8515a18f18c55..0890faa33b507 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala @@ -17,14 +17,13 @@ package org.apache.spark.sql.execution +import java.util.HashMap + import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ -/* Implicit conversions */ -import org.apache.spark.rdd.PartitionLocalRDDFunctions._ - /** * Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each * group. @@ -40,7 +39,7 @@ case class Aggregate( groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], child: SparkPlan)(@transient sc: SparkContext) - extends UnaryNode { + extends UnaryNode with NoBind { override def requiredChildDistribution = if (partial) { @@ -55,61 +54,149 @@ case class Aggregate( override def otherCopyArgs = sc :: Nil + // HACK: Generators don't correctly preserve their output through serializations so we grab + // out child's output attributes statically here. + val childOutput = child.output + def output = aggregateExpressions.map(_.toAttribute) - /* Replace all aggregate expressions with spark functions that will compute the result. */ - def createAggregateImplementations() = aggregateExpressions.map { agg => - val impl = agg transform { - case a: AggregateExpression => a.newInstance + /** + * An aggregate that needs to be computed for each row in a group. + * + * @param unbound Unbound version of this aggregate, used for result substitution. + * @param aggregate A bound copy of this aggregate used to create a new aggregation buffer. + * @param resultAttribute An attribute used to refer to the result of this aggregate in the final + * output. + */ + case class ComputedAggregate( + unbound: AggregateExpression, + aggregate: AggregateExpression, + resultAttribute: AttributeReference) + + /** A list of aggregates that need to be computed for each group. */ + @transient + lazy val computedAggregates = aggregateExpressions.flatMap { agg => + agg.collect { + case a: AggregateExpression => + ComputedAggregate( + a, + BindReferences.bindReference(a, childOutput).asInstanceOf[AggregateExpression], + AttributeReference(s"aggResult:$a", a.dataType, nullable = true)()) } + }.toArray + + /** The schema of the result of all aggregate evaluations */ + @transient + lazy val computedSchema = computedAggregates.map(_.resultAttribute) + + /** Creates a new aggregate buffer for a group. */ + def newAggregateBuffer(): Array[AggregateFunction] = { + val buffer = new Array[AggregateFunction](computedAggregates.length) + var i = 0 + while (i < computedAggregates.length) { + buffer(i) = computedAggregates(i).aggregate.newInstance() + i += 1 + } + buffer + } - val remainingAttributes = impl.collect { case a: Attribute => a } - // If any references exist that are not inside agg functions then the must be grouping exprs - // in this case we must rebind them to the grouping tuple. - if (remainingAttributes.nonEmpty) { - val unaliasedAggregateExpr = agg transform { case Alias(c, _) => c } - - // An exact match with a grouping expression - val exactGroupingExpr = groupingExpressions.indexOf(unaliasedAggregateExpr) match { - case -1 => None - case ordinal => Some(BoundReference(ordinal, Alias(impl, "AGGEXPR")().toAttribute)) - } + /** Named attributes used to substitute grouping attributes into the final result. */ + @transient + lazy val namedGroups = groupingExpressions.map { + case ne: NamedExpression => ne -> ne.toAttribute + case e => e -> Alias(e, s"groupingExpr:$e")().toAttribute + } - exactGroupingExpr.getOrElse( - sys.error(s"$agg is not in grouping expressions: $groupingExpressions")) - } else { - impl + /** + * A map of substitutions that are used to insert the aggregate expressions and grouping + * expression into the final result expression. + */ + @transient + lazy val resultMap = + (computedAggregates.map { agg => agg.unbound -> agg.resultAttribute} ++ namedGroups).toMap + + /** + * Substituted version of aggregateExpressions expressions which are used to compute final + * output rows given a group and the result of all aggregate computations. + */ + @transient + lazy val resultExpressions = aggregateExpressions.map { agg => + agg.transform { + case e: Expression if resultMap.contains(e) => resultMap(e) } } def execute() = attachTree(this, "execute") { - // TODO: If the child of it is an [[catalyst.execution.Exchange]], - // do not evaluate the groupingExpressions again since we have evaluated it - // in the [[catalyst.execution.Exchange]]. - val grouped = child.execute().mapPartitions { iter => - val buildGrouping = new Projection(groupingExpressions) - iter.map(row => (buildGrouping(row), row.copy())) - }.groupByKeyLocally() - - val result = grouped.map { case (group, rows) => - val aggImplementations = createAggregateImplementations() - - // Pull out all the functions so we can feed each row into them. - val aggFunctions = aggImplementations.flatMap(_ collect { case f: AggregateFunction => f }) - - rows.foreach { row => - aggFunctions.foreach(_.update(row)) + if (groupingExpressions.isEmpty) { + child.execute().mapPartitions { iter => + val buffer = newAggregateBuffer() + var currentRow: Row = null + while (iter.hasNext) { + currentRow = iter.next() + var i = 0 + while (i < buffer.length) { + buffer(i).update(currentRow) + i += 1 + } + } + val resultProjection = new Projection(resultExpressions, computedSchema) + val aggregateResults = new GenericMutableRow(computedAggregates.length) + + var i = 0 + while (i < buffer.length) { + aggregateResults(i) = buffer(i).eval(EmptyRow) + i += 1 + } + + Iterator(resultProjection(aggregateResults)) } - buildRow(aggImplementations.map(_.apply(group))) - } - - // TODO: THIS BREAKS PIPELINING, DOUBLE COMPUTES THE ANSWER, AND USES TOO MUCH MEMORY... - if (groupingExpressions.isEmpty && result.count == 0) { - // When there there is no output to the Aggregate operator, we still output an empty row. - val aggImplementations = createAggregateImplementations() - sc.makeRDD(buildRow(aggImplementations.map(_.apply(null))) :: Nil) } else { - result + child.execute().mapPartitions { iter => + val hashTable = new HashMap[Row, Array[AggregateFunction]] + val groupingProjection = new MutableProjection(groupingExpressions, childOutput) + + var currentRow: Row = null + while (iter.hasNext) { + currentRow = iter.next() + val currentGroup = groupingProjection(currentRow) + var currentBuffer = hashTable.get(currentGroup) + if (currentBuffer == null) { + currentBuffer = newAggregateBuffer() + hashTable.put(currentGroup.copy(), currentBuffer) + } + + var i = 0 + while (i < currentBuffer.length) { + currentBuffer(i).update(currentRow) + i += 1 + } + } + + new Iterator[Row] { + private[this] val hashTableIter = hashTable.entrySet().iterator() + private[this] val aggregateResults = new GenericMutableRow(computedAggregates.length) + private[this] val resultProjection = + new MutableProjection(resultExpressions, computedSchema ++ namedGroups.map(_._2)) + private[this] val joinedRow = new JoinedRow + + override final def hasNext: Boolean = hashTableIter.hasNext + + override final def next(): Row = { + val currentEntry = hashTableIter.next() + val currentGroup = currentEntry.getKey + val currentBuffer = currentEntry.getValue + + var i = 0 + while (i < currentBuffer.length) { + // Evaluating an aggregate buffer returns the result. No row is required since we + // already added all rows in the group using update. + aggregateResults(i) = currentBuffer(i).eval(EmptyRow) + i += 1 + } + resultProjection(joinedRow(aggregateResults, currentGroup)) + } + } + } } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 524e5022ee14b..ab2e62463764a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -41,7 +41,7 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { override def output = child.output override def execute() = child.execute().mapPartitions { iter => - iter.filter(condition.apply(_).asInstanceOf[Boolean]) + iter.filter(condition.eval(_).asInstanceOf[Boolean]) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 114bfbb719ee9..505ad0a2c77c1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -103,7 +103,7 @@ private[sql] object ParquetRelation { SLF4JBridgeHandler.install() for(name <- loggerNames) { val logger = Logger.getLogger(name) - logger.setParent(Logger.getGlobal) + logger.setParent(Logger.getLogger(Logger.GLOBAL_LOGGER_NAME)) logger.setUseParentHandlers(true) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index e5902c3cae381..7c6a642278226 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -58,4 +58,17 @@ class CachedTableSuite extends QueryTest { TestSQLContext.uncacheTable("testData") } } + + test("SELECT Star Cached Table") { + TestSQLContext.sql("SELECT * FROM testData").registerAsTable("selectStar") + TestSQLContext.cacheTable("selectStar") + TestSQLContext.sql("SELECT * FROM selectStar") + TestSQLContext.uncacheTable("selectStar") + } + + test("Self-join cached") { + TestSQLContext.cacheTable("testData") + TestSQLContext.sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key") + TestSQLContext.uncacheTable("testData") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 2524a37cbac13..be0f4a4c73b36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -119,8 +119,8 @@ class DslQuerySuite extends QueryTest { } test("inner join, where, multiple matches") { - val x = testData2.where('a === 1).subquery('x) - val y = testData2.where('a === 1).subquery('y) + val x = testData2.where('a === 1).as('x) + val y = testData2.where('a === 1).as('y) checkAnswer( x.join(y).where("x.a".attr === "y.a".attr), (1,1,1,1) :: @@ -131,8 +131,8 @@ class DslQuerySuite extends QueryTest { } test("inner join, no matches") { - val x = testData2.where('a === 1).subquery('x) - val y = testData2.where('a === 2).subquery('y) + val x = testData2.where('a === 1).as('x) + val y = testData2.where('a === 2).as('y) checkAnswer( x.join(y).where("x.a".attr === "y.a".attr), Nil) @@ -140,8 +140,8 @@ class DslQuerySuite extends QueryTest { test("big inner join, 4 matches per row") { val bigData = testData.unionAll(testData).unionAll(testData).unionAll(testData) - val bigDataX = bigData.subquery('x) - val bigDataY = bigData.subquery('y) + val bigDataX = bigData.as('x) + val bigDataY = bigData.as('y) checkAnswer( bigDataX.join(bigDataY).where("x.key".attr === "y.key".attr), @@ -181,8 +181,8 @@ class DslQuerySuite extends QueryTest { } test("full outer join") { - val left = upperCaseData.where('N <= 4).subquery('left) - val right = upperCaseData.where('N >= 3).subquery('right) + val left = upperCaseData.where('N <= 4).as('left) + val right = upperCaseData.where('N >= 3).as('right) checkAnswer( left.join(right, FullOuter, Some("left.N".attr === "right.N".attr)), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala index ca5c8b8eb63dc..e55648b8ed15a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala @@ -39,9 +39,9 @@ case class ExampleTGF(input: Seq[Attribute] = Seq('name, 'age)) extends Generato val Seq(nameAttr, ageAttr) = input - override def apply(input: Row): TraversableOnce[Row] = { - val name = nameAttr.apply(input) - val age = ageAttr.apply(input).asInstanceOf[Int] + override def eval(input: Row): TraversableOnce[Row] = { + val name = nameAttr.eval(input) + val age = ageAttr.eval(input).asInstanceOf[Int] Iterator( new GenericRow(Array[Any](s"$name is $age years old")), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index a62a3c4d02354..fc68d6c5620d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -56,8 +56,8 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { } test("self-join parquet files") { - val x = ParquetTestData.testData.subquery('x) - val y = ParquetTestData.testData.subquery('y) + val x = ParquetTestData.testData.as('x) + val y = ParquetTestData.testData.as('y) val query = x.join(y).where("x.myint".attr === "y.myint".attr) // Check to make sure that the attributes from either side of the join have unique expression diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 63f592cb4b441..a662da76ce25a 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -63,6 +63,10 @@ hive-exec ${hive.version} + + org.codehaus.jackson + jackson-mapper-asl + org.apache.hive hive-serde @@ -87,6 +91,30 @@ org.scalatest scalatest-maven-plugin + + + + org.apache.maven.plugins + maven-dependency-plugin + 2.4 + + + copy-dependencies + package + + copy-dependencies + + + + ${basedir}/../../lib_managed/jars + false + false + true + org.datanucleus + + + + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index ff8eaacded4c8..353458432b210 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -67,14 +67,13 @@ class LocalHiveContext(sc: SparkContext) extends HiveContext(sc) { class HiveContext(sc: SparkContext) extends SQLContext(sc) { self => - override def parseSql(sql: String): LogicalPlan = HiveQl.parseSql(sql) - override def executePlan(plan: LogicalPlan): this.QueryExecution = + override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } /** * Executes a query expressed in HiveQL using Spark, returning the result as a SchemaRDD. */ - def hql(hqlQuery: String): SchemaRDD = { + def hiveql(hqlQuery: String): SchemaRDD = { val result = new SchemaRDD(this, HiveQl.parseSql(hqlQuery)) // We force query optimization to happen right away instead of letting it happen lazily like // when using the query DSL. This is so DDL commands behave as expected. This is only @@ -83,6 +82,9 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { result } + /** An alias for `hiveql`. */ + def hql(hqlQuery: String): SchemaRDD = hiveql(hqlQuery) + // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. @transient protected val outputBuffer = new java.io.OutputStream { @@ -120,7 +122,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /* A catalyst metadata catalog that points to the Hive Metastore. */ @transient - override lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog { + override protected[sql] lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog { override def lookupRelation( databaseName: Option[String], tableName: String, @@ -132,7 +134,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /* An analyzer that uses the Hive metastore. */ @transient - override lazy val analyzer = new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) + override protected[sql] lazy val analyzer = + new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) /** * Runs the specified SQL query using Hive. @@ -214,14 +217,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } @transient - override val planner = hivePlanner + override protected[sql] val planner = hivePlanner @transient protected lazy val emptyResult = sparkContext.parallelize(Seq(new GenericRow(Array[Any]()): Row), 1) /** Extends QueryExecution with hive specific features. */ - abstract class QueryExecution extends super.QueryExecution { + protected[sql] abstract class QueryExecution extends super.QueryExecution { // TODO: Create mixin for the analyzer instead of overriding things here. override lazy val optimizedPlan = optimizer(catalog.PreInsertionCasts(catalog.CreateTables(analyzed))) @@ -294,5 +297,11 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { val asString = result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")).toSeq asString } + + override def simpleString: String = + logical match { + case _: NativeCommand => "" + case _ => executedPlan.toString + } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 0a6bea0162430..2fea9702954d7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -110,10 +110,10 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { val describedTable = "DESCRIBE (\\w+)".r - class SqlQueryExecution(sql: String) extends this.QueryExecution { - lazy val logical = HiveQl.parseSql(sql) - def hiveExec() = runSqlHive(sql) - override def toString = sql + "\n" + super.toString + protected[hive] class HiveQLQueryExecution(hql: String) extends this.QueryExecution { + lazy val logical = HiveQl.parseSql(hql) + def hiveExec() = runSqlHive(hql) + override def toString = hql + "\n" + super.toString } /** @@ -140,8 +140,8 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { case class TestTable(name: String, commands: (()=>Unit)*) - implicit class SqlCmd(sql: String) { - def cmd = () => new SqlQueryExecution(sql).stringResult(): Unit + protected[hive] implicit class SqlCmd(sql: String) { + def cmd = () => new HiveQLQueryExecution(sql).stringResult(): Unit } /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala index e2d9d8de2572a..821fb22112f87 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala @@ -106,7 +106,7 @@ case class HiveTableScan( } private def castFromString(value: String, dataType: DataType) = { - Cast(Literal(value), dataType).apply(null) + Cast(Literal(value), dataType).eval(null) } @transient @@ -134,7 +134,7 @@ case class HiveTableScan( // Only partitioned values are needed here, since the predicate has already been bound to // partition key attribute references. val row = new GenericRow(castedValues.toArray) - shouldKeep.apply(row).asInstanceOf[Boolean] + shouldKeep.eval(row).asInstanceOf[Boolean] } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 44901db3f963b..f9b437d435eba 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -190,8 +190,8 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd } // TODO: Finish input output types. - override def apply(input: Row): Any = { - val evaluatedChildren = children.map(_.apply(input)) + override def eval(input: Row): Any = { + val evaluatedChildren = children.map(_.eval(input)) // Wrap the function arguments in the expected types. val args = evaluatedChildren.zip(wrappers).map { case (arg, wrapper) => wrapper(arg) @@ -216,12 +216,12 @@ case class HiveGenericUdf( val dataType: DataType = inspectorToDataType(returnInspector) - override def apply(input: Row): Any = { + override def eval(input: Row): Any = { returnInspector // Make sure initialized. val args = children.map { v => new DeferredObject { override def prepare(i: Int) = {} - override def get(): AnyRef = wrap(v.apply(input)) + override def get(): AnyRef = wrap(v.eval(input)) } }.toArray unwrap(function.evaluate(args)) @@ -337,13 +337,16 @@ case class HiveGenericUdaf( type UDFType = AbstractGenericUDAFResolver + @transient protected lazy val resolver: AbstractGenericUDAFResolver = createFunction(name) + @transient protected lazy val objectInspector = { resolver.getEvaluator(children.map(_.dataType.toTypeInfo).toArray) .init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors.toArray) } + @transient protected lazy val inspectors = children.map(_.dataType).map(toInspector) def dataType: DataType = inspectorToDataType(objectInspector) @@ -403,7 +406,7 @@ case class HiveGenericUdtf( } } - override def apply(input: Row): TraversableOnce[Row] = { + override def eval(input: Row): TraversableOnce[Row] = { outputInspectors // Make sure initialized. val inputProjection = new Projection(children) @@ -457,7 +460,7 @@ case class HiveUdafFunction( private val buffer = function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] - override def apply(input: Row): Any = unwrapData(function.evaluate(buffer), returnInspector) + override def eval(input: Row): Any = unwrapData(function.evaluate(buffer), returnInspector) @transient val inputProjection = new Projection(exprs) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 18654b308d234..3cc4562a88d66 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -125,7 +125,7 @@ abstract class HiveComparisonTest } protected def prepareAnswer( - hiveQuery: TestHive.type#SqlQueryExecution, + hiveQuery: TestHive.type#HiveQLQueryExecution, answer: Seq[String]): Seq[String] = { val orderedAnswer = hiveQuery.logical match { // Clean out non-deterministic time schema info. @@ -227,7 +227,7 @@ abstract class HiveComparisonTest try { // MINOR HACK: You must run a query before calling reset the first time. - TestHive.sql("SHOW TABLES") + TestHive.hql("SHOW TABLES") if (reset) { TestHive.reset() } val hiveCacheFiles = queryList.zipWithIndex.map { @@ -256,7 +256,7 @@ abstract class HiveComparisonTest hiveCachedResults } else { - val hiveQueries = queryList.map(new TestHive.SqlQueryExecution(_)) + val hiveQueries = queryList.map(new TestHive.HiveQLQueryExecution(_)) // Make sure we can at least parse everything before attempting hive execution. hiveQueries.foreach(_.logical) val computedResults = (queryList.zipWithIndex, hiveQueries, hiveCacheFiles).zipped.map { @@ -302,7 +302,7 @@ abstract class HiveComparisonTest // Run w/ catalyst val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => - val query = new TestHive.SqlQueryExecution(queryString) + val query = new TestHive.HiveQLQueryExecution(queryString) try { (query, prepareAnswer(query, query.stringResult())) } catch { case e: Exception => val errorMessage = @@ -359,7 +359,7 @@ abstract class HiveComparisonTest // When we encounter an error we check to see if the environment is still okay by running a simple query. // If this fails then we halt testing since something must have gone seriously wrong. try { - new TestHive.SqlQueryExecution("SELECT key FROM src").stringResult() + new TestHive.HiveQLQueryExecution("SELECT key FROM src").stringResult() TestHive.runSqlHive("SELECT key FROM src") } catch { case e: Exception => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index c184ebe288af4..a09667ac84b01 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -23,6 +23,16 @@ import org.apache.spark.sql.hive.TestHive._ * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. */ class HiveQuerySuite extends HiveComparisonTest { + + test("Query expressed in SQL") { + assert(sql("SELECT 1").collect() === Array(Seq(1))) + } + + test("Query expressed in HiveQL") { + hql("FROM src SELECT key").collect() + hiveql("FROM src SELECT key").collect() + } + createQueryTest("Simple Average", "SELECT AVG(key) FROM src") @@ -133,7 +143,11 @@ class HiveQuerySuite extends HiveComparisonTest { "SELECT * FROM src LATERAL VIEW explode(map(key+3,key+4)) D as k, v") test("sampling") { - sql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s") + hql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s") } + test("SchemaRDD toString") { + hql("SHOW TABLES").toString + hql("SELECT * FROM src").toString + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index 40c4e23f90fb8..8883e5b16d4da 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -56,7 +56,7 @@ class HiveResolutionSuite extends HiveComparisonTest { TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2)) :: Nil) .registerAsTable("caseSensitivityTest") - sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") + hql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") } /** diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 1318ac1968dad..d9ccb93e23923 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -136,7 +136,7 @@ class PruningSuite extends HiveComparisonTest { expectedScannedColumns: Seq[String], expectedPartValues: Seq[Seq[String]]) = { test(s"$testCaseName - pruning test") { - val plan = new TestHive.SqlQueryExecution(sql).executedPlan + val plan = new TestHive.HiveQLQueryExecution(sql).executedPlan val actualOutputColumns = plan.output.map(_.name) val (actualScannedColumns, actualPartValues) = plan.collect { case p @ HiveTableScan(columns, relation, _) => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index 314ca48ad8f6a..aade62eb8f84e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -57,34 +57,34 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft } test("SELECT on Parquet table") { - val rdd = sql("SELECT * FROM testsource").collect() + val rdd = hql("SELECT * FROM testsource").collect() assert(rdd != null) assert(rdd.forall(_.size == 6)) } test("Simple column projection + filter on Parquet table") { - val rdd = sql("SELECT myboolean, mylong FROM testsource WHERE myboolean=true").collect() + val rdd = hql("SELECT myboolean, mylong FROM testsource WHERE myboolean=true").collect() assert(rdd.size === 5, "Filter returned incorrect number of rows") assert(rdd.forall(_.getBoolean(0)), "Filter returned incorrect Boolean field value") } test("Converting Hive to Parquet Table via saveAsParquetFile") { - sql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath) + hql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath) parquetFile(dirname.getAbsolutePath).registerAsTable("ptable") - val rddOne = sql("SELECT * FROM src").collect().sortBy(_.getInt(0)) - val rddTwo = sql("SELECT * from ptable").collect().sortBy(_.getInt(0)) + val rddOne = hql("SELECT * FROM src").collect().sortBy(_.getInt(0)) + val rddTwo = hql("SELECT * from ptable").collect().sortBy(_.getInt(0)) compareRDDs(rddOne, rddTwo, "src (Hive)", Seq("key:Int", "value:String")) } test("INSERT OVERWRITE TABLE Parquet table") { - sql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath) + hql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath) parquetFile(dirname.getAbsolutePath).registerAsTable("ptable") // let's do three overwrites for good measure - sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() - sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() - sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() - val rddCopy = sql("SELECT * FROM ptable").collect() - val rddOrig = sql("SELECT * FROM testsource").collect() + hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() + hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() + hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() + val rddCopy = hql("SELECT * FROM ptable").collect() + val rddOrig = hql("SELECT * FROM testsource").collect() assert(rddCopy.size === rddOrig.size, "INSERT OVERWRITE changed size of table??") compareRDDs(rddOrig, rddCopy, "testsource", ParquetTestData.testSchemaFieldNames) } @@ -93,13 +93,13 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft createParquetFile(dirname.getAbsolutePath, ("key", IntegerType), ("value", StringType)) .registerAsTable("tmp") val rddCopy = - sql("INSERT INTO TABLE tmp SELECT * FROM src") + hql("INSERT INTO TABLE tmp SELECT * FROM src") .collect() .sortBy[Int](_.apply(0) match { case x: Int => x case _ => 0 }) - val rddOrig = sql("SELECT * FROM src") + val rddOrig = hql("SELECT * FROM src") .collect() .sortBy(_.getInt(0)) compareRDDs(rddOrig, rddCopy, "src (Hive)", Seq("key:Int", "value:String")) @@ -108,22 +108,22 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft test("Appending to Parquet table") { createParquetFile(dirname.getAbsolutePath, ("key", IntegerType), ("value", StringType)) .registerAsTable("tmpnew") - sql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect() - sql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect() - sql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect() - val rddCopies = sql("SELECT * FROM tmpnew").collect() - val rddOrig = sql("SELECT * FROM src").collect() + hql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect() + hql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect() + hql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect() + val rddCopies = hql("SELECT * FROM tmpnew").collect() + val rddOrig = hql("SELECT * FROM src").collect() assert(rddCopies.size === 3 * rddOrig.size, "number of copied rows via INSERT INTO did not match correct number") } test("Appending to and then overwriting Parquet table") { createParquetFile(dirname.getAbsolutePath, ("key", IntegerType), ("value", StringType)) .registerAsTable("tmp") - sql("INSERT INTO TABLE tmp SELECT * FROM src").collect() - sql("INSERT INTO TABLE tmp SELECT * FROM src").collect() - sql("INSERT OVERWRITE TABLE tmp SELECT * FROM src").collect() - val rddCopies = sql("SELECT * FROM tmp").collect() - val rddOrig = sql("SELECT * FROM src").collect() + hql("INSERT INTO TABLE tmp SELECT * FROM src").collect() + hql("INSERT INTO TABLE tmp SELECT * FROM src").collect() + hql("INSERT OVERWRITE TABLE tmp SELECT * FROM src").collect() + val rddCopies = hql("SELECT * FROM tmp").collect() + val rddOrig = hql("SELECT * FROM src").collect() assert(rddCopies.size === rddOrig.size, "INSERT OVERWRITE did not actually overwrite") } diff --git a/streaming/pom.xml b/streaming/pom.xml index 1953cc6883378..93b1c5a37aff9 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -96,7 +96,6 @@ org.apache.maven.plugins maven-jar-plugin - 2.2 diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 7df206241beb6..389b23d4d5e4b 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -144,8 +144,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } - - test("actor input stream") { + // TODO: This test makes assumptions about Thread.sleep() and is flaky + ignore("actor input stream") { // Start the server val testServer = new TestServer() val port = testServer.port diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 981e8b05f602d..3469b7decedf6 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -81,7 +81,8 @@ class ExecutorRunnable( credentials.writeTokenStorageToStream(dob) ctx.setContainerTokens(ByteBuffer.wrap(dob.getData())) - val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores) + val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, + localResources.contains(ClientBase.LOG4J_PROP)) logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) diff --git a/yarn/common/src/main/resources/log4j-spark-container.properties b/yarn/common/src/main/resources/log4j-spark-container.properties new file mode 100644 index 0000000000000..a1e37a0be27dd --- /dev/null +++ b/yarn/common/src/main/resources/log4j-spark-container.properties @@ -0,0 +1,24 @@ +# +# Licensed 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. See accompanying LICENSE file. + +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + +# Settings to quiet third party logs that are too verbose +log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO +log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 6568003bf1008..eb42922aea228 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -266,11 +266,11 @@ trait ClientBase extends Logging { localResources: HashMap[String, LocalResource], stagingDir: String): HashMap[String, String] = { logInfo("Setting up the launch environment") - val log4jConfLocalRes = localResources.getOrElse(ClientBase.LOG4J_PROP, null) val env = new HashMap[String, String]() - ClientBase.populateClasspath(yarnConf, sparkConf, log4jConfLocalRes != null, env) + ClientBase.populateClasspath(yarnConf, sparkConf, localResources.contains(ClientBase.LOG4J_PROP), + env) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() @@ -344,15 +344,13 @@ trait ClientBase extends Logging { JAVA_OPTS += " " + env("SPARK_JAVA_OPTS") } - // Command for the ApplicationMaster - var javaCommand = "java" - val javaHome = System.getenv("JAVA_HOME") - if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) { - javaCommand = Environment.JAVA_HOME.$() + "/bin/java" + if (!localResources.contains(ClientBase.LOG4J_PROP)) { + JAVA_OPTS += " " + YarnSparkHadoopUtil.getLoggingArgsForContainerCommandLine() } + // Command for the ApplicationMaster val commands = List[String]( - javaCommand + + Environment.JAVA_HOME.$() + "/bin/java" + " -server " + JAVA_OPTS + " " + args.amClass + diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index da0a6f74efcd5..b3696c5fe7183 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -50,7 +50,8 @@ trait ExecutorRunnableUtil extends Logging { slaveId: String, hostname: String, executorMemory: Int, - executorCores: Int) = { + executorCores: Int, + userSpecifiedLogFile: Boolean) = { // Extra options for the JVM var JAVA_OPTS = "" // Set the JVM memory @@ -63,6 +64,10 @@ trait ExecutorRunnableUtil extends Logging { JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " " + if (!userSpecifiedLogFile) { + JAVA_OPTS += " " + YarnSparkHadoopUtil.getLoggingArgsForContainerCommandLine() + } + // Commenting it out for now - so that people can refer to the properties if required. Remove // it once cpuset version is pushed out. // The context is, default gc for server class machines end up using all cores to do gc - hence @@ -88,13 +93,8 @@ trait ExecutorRunnableUtil extends Logging { } */ - var javaCommand = "java" - val javaHome = System.getenv("JAVA_HOME") - if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) { - javaCommand = Environment.JAVA_HOME.$() + "/bin/java" - } - - val commands = List[String](javaCommand + + val commands = List[String]( + Environment.JAVA_HOME.$() + "/bin/java" + " -server " + // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. // Not killing the task leaves various aspects of the executor and (to some extent) the jvm in diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 4c6e1dcd6dac3..314a7550ada71 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.conf.Configuration import org.apache.spark.deploy.SparkHadoopUtil @@ -67,3 +68,9 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } } + +object YarnSparkHadoopUtil { + def getLoggingArgsForContainerCommandLine(): String = { + "-Dlog4j.configuration=log4j-spark-container.properties" + } +} diff --git a/yarn/pom.xml b/yarn/pom.xml index 35e31760c1f02..3342cb65edcd1 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -167,6 +167,12 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + + ../common/src/main/resources + + diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 53c403f7d0913..81d9d1b5c9280 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -78,7 +78,8 @@ class ExecutorRunnable( credentials.writeTokenStorageToStream(dob) ctx.setTokens(ByteBuffer.wrap(dob.getData())) - val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores) + val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, + localResources.contains(ClientBase.LOG4J_PROP)) logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands)
Storage LevelMeaning
MEMORY_AND_DISK_SER Similar to MEMORY_ONLY_SER, but spill partitions that don't fit in memory to disk instead of recomputing them - on the fly each time they're needed. Similar to MEMORY_ONLY_SER, but spill partitions that don't fit in memory to disk instead of + recomputing them on the fly each time they're needed.
OFF_HEAP Store RDD in a serialized format in Tachyon. + This is generally more space-efficient than deserialized objects, especially when using a + fast serializer, but more CPU-intensive to read. + This also significantly reduces the overheads of GC. +
DISK_ONLY