diff --git a/.gitignore b/.gitignore index 5b56a67c883e6..a4ec12ca6b53f 100644 --- a/.gitignore +++ b/.gitignore @@ -51,10 +51,10 @@ unit-tests.log rat-results.txt scalastyle.txt conf/*.conf +scalastyle-output.xml # For Hive metastore_db/ metastore/ warehouse/ TempStatsStore/ -sql/hive-thriftserver/test_warehouses diff --git a/assembly/pom.xml b/assembly/pom.xml index 703f15925bc44..567a8dd2a0d94 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -165,16 +165,6 @@ - - hive-thriftserver - - - org.apache.spark - spark-hive-thriftserver_${scala.binary.version} - ${project.version} - - - spark-ganglia-lgpl diff --git a/bagel/pom.xml b/bagel/pom.xml index bd51b112e26fa..90c4b095bb611 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-bagel_2.10 - bagel + bagel jar Spark Project Bagel diff --git a/bin/beeline b/bin/beeline deleted file mode 100755 index 09fe366c609fa..0000000000000 --- a/bin/beeline +++ /dev/null @@ -1,45 +0,0 @@ -#!/usr/bin/env bash - -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" - -# Find the java binary -if [ -n "${JAVA_HOME}" ]; then - RUNNER="${JAVA_HOME}/bin/java" -else - if [ `command -v java` ]; then - RUNNER="java" - else - echo "JAVA_HOME is not set" >&2 - exit 1 - fi -fi - -# Compute classpath using external script -classpath_output=$($FWDIR/bin/compute-classpath.sh) -if [[ "$?" != "0" ]]; then - echo "$classpath_output" - exit 1 -else - CLASSPATH=$classpath_output -fi - -CLASS="org.apache.hive.beeline.BeeLine" -exec "$RUNNER" -cp "$CLASSPATH" $CLASS "$@" diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 16b794a1592e8..e81e8c060cb98 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -52,7 +52,6 @@ if [ -n "$SPARK_PREPEND_CLASSES" ]; then CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/hive-thriftserver/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SCALA_VERSION/classes" fi diff --git a/bin/spark-shell b/bin/spark-shell index 756c8179d12b6..850e9507ec38f 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -46,11 +46,11 @@ function main(){ # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main spark-shell "$@" + $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main stty icanon echo > /dev/null 2>&1 else export SPARK_SUBMIT_OPTS - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main spark-shell "$@" + $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main fi } diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd index b56d69801171c..4b9708a8c03f3 100755 --- a/bin/spark-shell.cmd +++ b/bin/spark-shell.cmd @@ -19,4 +19,4 @@ rem set SPARK_HOME=%~dp0.. -cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell --class org.apache.spark.repl.Main %* +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell %* --class org.apache.spark.repl.Main diff --git a/bin/spark-sql b/bin/spark-sql deleted file mode 100755 index bba7f897b19bc..0000000000000 --- a/bin/spark-sql +++ /dev/null @@ -1,36 +0,0 @@ -#!/usr/bin/env bash - -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# -# Shell script for starting the Spark SQL CLI - -# Enter posix mode for bash -set -o posix - -# Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" - -if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - echo "Usage: ./sbin/spark-sql [options]" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit 0 -fi - -CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" -exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal $@ diff --git a/core/pom.xml b/core/pom.xml index a24743495b0e1..1054cec4d77bb 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-core_2.10 - core + core jar Spark Project Core diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 1d640579efe77..ff0ca11749d42 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -55,10 +55,7 @@ case class Aggregator[K, V, C] ( combiners.iterator } else { val combiners = new ExternalAppendOnlyMap[K, V, C](createCombiner, mergeValue, mergeCombiners) - while (iter.hasNext) { - val pair = iter.next() - combiners.insert(pair._1, pair._2) - } + combiners.insertAll(iter) // TODO: Make this non optional in a future release Option(context).foreach(c => c.taskMetrics.memoryBytesSpilled = combiners.memoryBytesSpilled) Option(context).foreach(c => c.taskMetrics.diskBytesSpilled = combiners.diskBytesSpilled) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 8f867686a0443..5ddda4d6953fa 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -17,9 +17,9 @@ package org.apache.spark -import scala.collection.mutable.{ArrayBuffer, HashSet} +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer -import org.apache.spark.executor.InputMetrics import org.apache.spark.rdd.RDD import org.apache.spark.storage._ @@ -30,7 +30,7 @@ import org.apache.spark.storage._ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { /** Keys of RDD partitions that are being computed/loaded. */ - private val loading = new HashSet[RDDBlockId]() + private val loading = new mutable.HashSet[RDDBlockId] /** Gets or computes an RDD partition. Used by RDD.iterator() when an RDD is cached. */ def getOrCompute[T]( @@ -118,21 +118,29 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { } /** - * Cache the values of a partition, keeping track of any updates in the storage statuses - * of other blocks along the way. + * Cache the values of a partition, keeping track of any updates in the storage statuses of + * other blocks along the way. + * + * The effective storage level refers to the level that actually specifies BlockManager put + * behavior, not the level originally specified by the user. This is mainly for forcing a + * MEMORY_AND_DISK partition to disk if there is not enough room to unroll the partition, + * while preserving the the original semantics of the RDD as specified by the application. */ private def putInBlockManager[T]( key: BlockId, values: Iterator[T], - storageLevel: StorageLevel, - updatedBlocks: ArrayBuffer[(BlockId, BlockStatus)]): Iterator[T] = { - - if (!storageLevel.useMemory) { - /* This RDD is not to be cached in memory, so we can just pass the computed values - * as an iterator directly to the BlockManager, rather than first fully unrolling - * it in memory. The latter option potentially uses much more memory and risks OOM - * exceptions that can be avoided. */ - updatedBlocks ++= blockManager.put(key, values, storageLevel, tellMaster = true) + level: StorageLevel, + updatedBlocks: ArrayBuffer[(BlockId, BlockStatus)], + effectiveStorageLevel: Option[StorageLevel] = None): Iterator[T] = { + + val putLevel = effectiveStorageLevel.getOrElse(level) + if (!putLevel.useMemory) { + /* + * This RDD is not to be cached in memory, so we can just pass the computed values as an + * iterator directly to the BlockManager rather than first fully unrolling it in memory. + */ + updatedBlocks ++= + blockManager.putIterator(key, values, level, tellMaster = true, effectiveStorageLevel) blockManager.get(key) match { case Some(v) => v.data.asInstanceOf[Iterator[T]] case None => @@ -140,14 +148,36 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { throw new BlockException(key, s"Block manager failed to return cached value for $key!") } } else { - /* This RDD is to be cached in memory. In this case we cannot pass the computed values + /* + * This RDD is to be cached in memory. In this case we cannot pass the computed values * to the BlockManager as an iterator and expect to read it back later. This is because - * we may end up dropping a partition from memory store before getting it back, e.g. - * when the entirety of the RDD does not fit in memory. */ - val elements = new ArrayBuffer[Any] - elements ++= values - updatedBlocks ++= blockManager.put(key, elements, storageLevel, tellMaster = true) - elements.iterator.asInstanceOf[Iterator[T]] + * we may end up dropping a partition from memory store before getting it back. + * + * In addition, we must be careful to not unroll the entire partition in memory at once. + * Otherwise, we may cause an OOM exception if the JVM does not have enough space for this + * single partition. Instead, we unroll the values cautiously, potentially aborting and + * dropping the partition to disk if applicable. + */ + blockManager.memoryStore.unrollSafely(key, values, updatedBlocks) match { + case Left(arr) => + // We have successfully unrolled the entire partition, so cache it in memory + updatedBlocks ++= + blockManager.putArray(key, arr, level, tellMaster = true, effectiveStorageLevel) + arr.iterator.asInstanceOf[Iterator[T]] + case Right(it) => + // There is not enough space to cache this partition in memory + logWarning(s"Not enough space to cache partition $key in memory! " + + s"Free memory is ${blockManager.memoryStore.freeMemory} bytes.") + val returnValues = it.asInstanceOf[Iterator[T]] + if (putLevel.useDisk) { + logWarning(s"Persisting partition $key to disk instead.") + val diskOnlyLevel = StorageLevel(useDisk = true, useMemory = false, + useOffHeap = false, deserialized = false, putLevel.replication) + putInBlockManager[T](key, returnValues, level, updatedBlocks, Some(diskOnlyLevel)) + } else { + returnValues + } + } } } diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 09a60571238ea..f010c03223ef4 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -19,6 +19,7 @@ package org.apache.spark import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.SortOrder.SortOrder import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleHandle @@ -62,7 +63,8 @@ class ShuffleDependency[K, V, C]( val serializer: Option[Serializer] = None, val keyOrdering: Option[Ordering[K]] = None, val aggregator: Option[Aggregator[K, V, C]] = None, - val mapSideCombine: Boolean = false) + val mapSideCombine: Boolean = false, + val sortOrder: Option[SortOrder] = None) extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) { val shuffleId: Int = rdd.context.newShuffleId() diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 8f70744d804d9..6ee731b22c03c 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -67,7 +67,7 @@ class SparkEnv ( val metricsSystem: MetricsSystem, val conf: SparkConf) extends Logging { - // A mapping of thread ID to amount of memory used for shuffle in bytes + // A mapping of thread ID to amount of memory, in bytes, used for shuffle aggregations // All accesses should be manually synchronized val shuffleMemoryMap = mutable.HashMap[Long, Long]() 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 1e0493c4855e0..8a5f8088a05ca 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 @@ -34,7 +34,7 @@ import org.apache.spark._ import org.apache.spark.SparkContext.{DoubleAccumulatorParam, IntAccumulatorParam} import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{EmptyRDD, RDD} /** * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns @@ -112,6 +112,9 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def startTime: java.lang.Long = sc.startTime + /** The version of Spark on which this application is running. */ + def version: String = sc.version + /** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */ def defaultParallelism: java.lang.Integer = sc.defaultParallelism @@ -132,6 +135,13 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices) } + /** Get an RDD that has no partitions or elements. */ + def emptyRDD[T]: JavaRDD[T] = { + implicit val ctag: ClassTag[T] = fakeClassTag + JavaRDD.fromRDD(new EmptyRDD[T](sc)) + } + + /** Distribute a local Scala collection to form an RDD. */ def parallelize[T](list: java.util.List[T]): JavaRDD[T] = parallelize(list, sc.defaultParallelism) 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 d6b0988641a97..d87783efd2d01 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 @@ -37,8 +37,8 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils -private[spark] class PythonRDD[T: ClassTag]( - parent: RDD[T], +private[spark] class PythonRDD( + parent: RDD[_], command: Array[Byte], envVars: JMap[String, String], pythonIncludes: JList[String], 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 c9cec33ebaa66..3b5642b6caa36 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -46,10 +46,6 @@ object SparkSubmit { private val CLUSTER = 2 private val ALL_DEPLOY_MODES = CLIENT | CLUSTER - // A special jar name that indicates the class being run is inside of Spark itself, and therefore - // no user jar is needed. - private val SPARK_INTERNAL = "spark-internal" - // Special primary resource names that represent shells rather than application jars. private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" @@ -261,9 +257,7 @@ object SparkSubmit { // In yarn-cluster mode, use yarn.Client as a wrapper around the user class if (clusterManager == YARN && deployMode == CLUSTER) { childMainClass = "org.apache.spark.deploy.yarn.Client" - if (args.primaryResource != SPARK_INTERNAL) { - childArgs += ("--jar", args.primaryResource) - } + childArgs += ("--jar", args.primaryResource) childArgs += ("--class", args.mainClass) if (args.childArgs != null) { args.childArgs.foreach { arg => childArgs += ("--arg", arg) } @@ -338,7 +332,7 @@ object SparkSubmit { * Return whether the given primary resource represents a user jar. */ private def isUserJar(primaryResource: String): Boolean = { - !isShell(primaryResource) && !isPython(primaryResource) && !isInternal(primaryResource) + !isShell(primaryResource) && !isPython(primaryResource) } /** @@ -355,10 +349,6 @@ object SparkSubmit { primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL } - private[spark] def isInternal(primaryResource: String): Boolean = { - primaryResource == SPARK_INTERNAL - } - /** * Merge a sequence of comma-separated file lists, some of which may be null to indicate * no files, into a single comma-separated string. diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 01d0ae541a66b..3ab67a43a3b55 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -204,9 +204,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { /** Fill in values by parsing user options. */ private def parseOpts(opts: Seq[String]): Unit = { - var inSparkOpts = true - // Delineates parsing of Spark options from parsing of user options. + var inSparkOpts = true parse(opts) def parse(opts: Seq[String]): Unit = opts match { @@ -319,7 +318,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { SparkSubmit.printErrorAndExit(errMessage) case v => primaryResource = - if (!SparkSubmit.isShell(v) && !SparkSubmit.isInternal(v)) { + if (!SparkSubmit.isShell(v)) { Utils.resolveURI(v).toString } else { v diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index b16133b20cc02..3b69bc4ca4142 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -266,11 +266,13 @@ private[spark] class Executor( } } } finally { - // TODO: Unregister shuffle memory only for ResultTask + // Release memory used by this thread for shuffles val shuffleMemoryMap = env.shuffleMemoryMap shuffleMemoryMap.synchronized { shuffleMemoryMap.remove(Thread.currentThread().getId) } + // Release memory used by this thread for unrolling blocks + env.blockManager.memoryStore.releaseUnrollMemoryForThisThread() runningTasks.remove(taskId) } } diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 8a1cdb812962e..566e8a4aaa1d2 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -62,13 +62,15 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, conf.getInt("spark.core.connection.handler.threads.min", 20), conf.getInt("spark.core.connection.handler.threads.max", 60), conf.getInt("spark.core.connection.handler.threads.keepalive", 60), TimeUnit.SECONDS, - new LinkedBlockingDeque[Runnable]()) + new LinkedBlockingDeque[Runnable](), + Utils.namedThreadFactory("handle-message-executor")) private val handleReadWriteExecutor = new ThreadPoolExecutor( conf.getInt("spark.core.connection.io.threads.min", 4), conf.getInt("spark.core.connection.io.threads.max", 32), conf.getInt("spark.core.connection.io.threads.keepalive", 60), TimeUnit.SECONDS, - new LinkedBlockingDeque[Runnable]()) + new LinkedBlockingDeque[Runnable](), + Utils.namedThreadFactory("handle-read-write-executor")) // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : // which should be executed asap @@ -76,7 +78,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, conf.getInt("spark.core.connection.connect.threads.min", 1), conf.getInt("spark.core.connection.connect.threads.max", 8), conf.getInt("spark.core.connection.connect.threads.keepalive", 60), TimeUnit.SECONDS, - new LinkedBlockingDeque[Runnable]()) + new LinkedBlockingDeque[Runnable](), + Utils.namedThreadFactory("handle-connect-executor")) private val serverChannel = ServerSocketChannel.open() // used to track the SendingConnections waiting to do SASL negotiation diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 7d96089e52ab9..6388ef82cc5db 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -154,11 +154,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: map.iterator.asInstanceOf[Iterator[(K, Array[Iterable[_]])]]) } else { val map = createExternalMap(numRdds) - rddIterators.foreach { case (it, depNum) => - while (it.hasNext) { - val kv = it.next() - map.insert(kv._1, new CoGroupValue(kv._2, depNum)) - } + for ((it, depNum) <- rddIterators) { + map.insertAll(it.map(pair => (pair._1, new CoGroupValue(pair._2, depNum)))) } context.taskMetrics.memoryBytesSpilled = map.memoryBytesSpilled context.taskMetrics.diskBytesSpilled = map.diskBytesSpilled diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index f1f4b4324edfd..afd7075f686b9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -57,14 +57,13 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, */ def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size): RDD[P] = { val part = new RangePartitioner(numPartitions, self, ascending) - val shuffled = new ShuffledRDD[K, V, V, P](self, part).setKeyOrdering(ordering) - shuffled.mapPartitions(iter => { - val buf = iter.toArray - if (ascending) { - buf.sortWith((x, y) => ordering.lt(x._1, y._1)).iterator - } else { - buf.sortWith((x, y) => ordering.gt(x._1, y._1)).iterator - } - }, preservesPartitioning = true) + new ShuffledRDD[K, V, V, P](self, part) + .setKeyOrdering(ordering) + .setSortOrder(if (ascending) SortOrder.ASCENDING else SortOrder.DESCENDING) } } + +private[spark] object SortOrder extends Enumeration { + type SortOrder = Value + val ASCENDING, DESCENDING = Value +} diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index bf02f68d0d3d3..da4a8c3dc22b1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -21,6 +21,7 @@ import scala.reflect.ClassTag import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.SortOrder.SortOrder import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -51,6 +52,8 @@ class ShuffledRDD[K, V, C, P <: Product2[K, C] : ClassTag]( private var mapSideCombine: Boolean = false + private var sortOrder: Option[SortOrder] = None + /** Set a serializer for this RDD's shuffle, or null to use the default (spark.serializer) */ def setSerializer(serializer: Serializer): ShuffledRDD[K, V, C, P] = { this.serializer = Option(serializer) @@ -75,8 +78,15 @@ class ShuffledRDD[K, V, C, P <: Product2[K, C] : ClassTag]( this } + /** Set sort order for RDD's sorting. */ + def setSortOrder(sortOrder: SortOrder): ShuffledRDD[K, V, C, P] = { + this.sortOrder = Option(sortOrder) + this + } + override def getDependencies: Seq[Dependency[_]] = { - List(new ShuffleDependency(prev, part, serializer, keyOrdering, aggregator, mapSideCombine)) + List(new ShuffleDependency(prev, part, serializer, + keyOrdering, aggregator, mapSideCombine, sortOrder)) } override val partitioner = Some(part) 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 acb4c4946eded..dc6142ab79d03 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -85,12 +85,9 @@ class DAGScheduler( private val nextStageId = new AtomicInteger(0) private[scheduler] val jobIdToStageIds = new HashMap[Int, HashSet[Int]] - private[scheduler] val stageIdToJobIds = new HashMap[Int, HashSet[Int]] private[scheduler] val stageIdToStage = new HashMap[Int, Stage] 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[scheduler] val stageToInfos = new HashMap[Stage, StageInfo] // Stages we need to run whose parents aren't done private[scheduler] val waitingStages = new HashSet[Stage] @@ -101,9 +98,6 @@ class DAGScheduler( // Stages that must be resubmitted due to fetch failures private[scheduler] val failedStages = new HashSet[Stage] - // Missing tasks from each stage - private[scheduler] val pendingTasks = new HashMap[Stage, HashSet[Task[_]]] - private[scheduler] val activeJobs = new HashSet[ActiveJob] // Contains the locations that each RDD's partitions are cached on @@ -223,7 +217,6 @@ class DAGScheduler( new Stage(id, rdd, numTasks, shuffleDep, getParentStages(rdd, jobId), jobId, callSite) stageIdToStage(id) = stage updateJobIdStageIdMaps(jobId, stage) - stageToInfos(stage) = StageInfo.fromStage(stage) stage } @@ -315,13 +308,12 @@ class DAGScheduler( */ private def updateJobIdStageIdMaps(jobId: Int, stage: Stage) { def updateJobIdStageIdMapsList(stages: List[Stage]) { - if (!stages.isEmpty) { + if (stages.nonEmpty) { val s = stages.head - stageIdToJobIds.getOrElseUpdate(s.id, new HashSet[Int]()) += jobId + s.jobIds += jobId jobIdToStageIds.getOrElseUpdate(jobId, new HashSet[Int]()) += s.id - val parents = getParentStages(s.rdd, jobId) - val parentsWithoutThisJobId = parents.filter(p => - !stageIdToJobIds.get(p.id).exists(_.contains(jobId))) + val parents: List[Stage] = getParentStages(s.rdd, jobId) + val parentsWithoutThisJobId = parents.filter { ! _.jobIds.contains(jobId) } updateJobIdStageIdMapsList(parentsWithoutThisJobId ++ stages.tail) } } @@ -333,16 +325,15 @@ class DAGScheduler( * handle cancelling tasks or notifying the SparkListener about finished jobs/stages/tasks. * * @param job The job whose state to cleanup. - * @param resultStage Specifies the result stage for the job; if set to None, this method - * searches resultStagesToJob to find and cleanup the appropriate result stage. */ - private def cleanupStateForJobAndIndependentStages(job: ActiveJob, resultStage: Option[Stage]) { + private def cleanupStateForJobAndIndependentStages(job: ActiveJob) { val registeredStages = jobIdToStageIds.get(job.jobId) if (registeredStages.isEmpty || registeredStages.get.isEmpty) { logError("No stages registered for job " + job.jobId) } else { - stageIdToJobIds.filterKeys(stageId => registeredStages.get.contains(stageId)).foreach { - case (stageId, jobSet) => + stageIdToStage.filterKeys(stageId => registeredStages.get.contains(stageId)).foreach { + case (stageId, stage) => + val jobSet = stage.jobIds if (!jobSet.contains(job.jobId)) { logError( "Job %d not registered for stage %d even though that stage was registered for the job" @@ -355,14 +346,9 @@ class DAGScheduler( logDebug("Removing running stage %d".format(stageId)) runningStages -= stage } - stageToInfos -= stage for ((k, v) <- shuffleToMapStage.find(_._2 == stage)) { shuffleToMapStage.remove(k) } - if (pendingTasks.contains(stage) && !pendingTasks(stage).isEmpty) { - logDebug("Removing pending status for stage %d".format(stageId)) - } - pendingTasks -= stage if (waitingStages.contains(stage)) { logDebug("Removing stage %d from waiting set.".format(stageId)) waitingStages -= stage @@ -374,7 +360,6 @@ class DAGScheduler( } // data structures based on StageId stageIdToStage -= stageId - stageIdToJobIds -= stageId ShuffleMapTask.removeStage(stageId) ResultTask.removeStage(stageId) @@ -393,19 +378,7 @@ class DAGScheduler( jobIdToStageIds -= job.jobId jobIdToActiveJob -= job.jobId activeJobs -= job - - if (resultStage.isEmpty) { - // Clean up result stages. - val resultStagesForJob = resultStageToJob.keySet.filter( - stage => resultStageToJob(stage).jobId == job.jobId) - if (resultStagesForJob.size != 1) { - logWarning( - s"${resultStagesForJob.size} result stages for job ${job.jobId} (expect exactly 1)") - } - resultStageToJob --= resultStagesForJob - } else { - resultStageToJob -= resultStage.get - } + job.finalStage.resultOfJob = None } /** @@ -591,9 +564,10 @@ class DAGScheduler( job.listener.jobFailed(exception) } finally { val s = job.finalStage - stageIdToJobIds -= s.id // clean up data structures that were populated for a local job, - stageIdToStage -= s.id // but that won't get cleaned up via the normal paths through - stageToInfos -= s // completion events or stage abort + // clean up data structures that were populated for a local job, + // but that won't get cleaned up via the normal paths through + // completion events or stage abort + stageIdToStage -= s.id jobIdToStageIds -= job.jobId listenerBus.post(SparkListenerJobEnd(job.jobId, jobResult)) } @@ -605,12 +579,8 @@ class DAGScheduler( // That should take care of at least part of the priority inversion problem with // cross-job dependencies. private def activeJobForStage(stage: Stage): Option[Int] = { - if (stageIdToJobIds.contains(stage.id)) { - val jobsThatUseStage: Array[Int] = stageIdToJobIds(stage.id).toArray.sorted - jobsThatUseStage.find(jobIdToActiveJob.contains) - } else { - None - } + val jobsThatUseStage: Array[Int] = stage.jobIds.toArray.sorted + jobsThatUseStage.find(jobIdToActiveJob.contains) } private[scheduler] def handleJobGroupCancelled(groupId: String) { @@ -642,9 +612,8 @@ class DAGScheduler( // is in the process of getting stopped. val stageFailedMessage = "Stage cancelled because SparkContext was shut down" runningStages.foreach { stage => - val info = stageToInfos(stage) - info.stageFailed(stageFailedMessage) - listenerBus.post(SparkListenerStageCompleted(info)) + stage.info.stageFailed(stageFailedMessage) + listenerBus.post(SparkListenerStageCompleted(stage.info)) } listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) } @@ -690,7 +659,7 @@ class DAGScheduler( } else { jobIdToActiveJob(jobId) = job activeJobs += job - resultStageToJob(finalStage) = job + finalStage.resultOfJob = Some(job) listenerBus.post(SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, properties)) submitStage(finalStage) @@ -710,7 +679,6 @@ class DAGScheduler( if (missing == Nil) { logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents") submitMissingTasks(stage, jobId.get) - runningStages += stage } else { for (parent <- missing) { submitStage(parent) @@ -728,8 +696,7 @@ class DAGScheduler( private def submitMissingTasks(stage: Stage, jobId: Int) { logDebug("submitMissingTasks(" + stage + ")") // Get our pending tasks and remember them in our pendingTasks entry - val myPending = pendingTasks.getOrElseUpdate(stage, new HashSet) - myPending.clear() + stage.pendingTasks.clear() var tasks = ArrayBuffer[Task[_]]() if (stage.isShuffleMap) { for (p <- 0 until stage.numPartitions if stage.outputLocs(p) == Nil) { @@ -738,7 +705,7 @@ class DAGScheduler( } } else { // This is a final stage; figure out its job's missing partitions - val job = resultStageToJob(stage) + val job = stage.resultOfJob.get for (id <- 0 until job.numPartitions if !job.finished(id)) { val partition = job.partitions(id) val locs = getPreferredLocs(stage.rdd, partition) @@ -753,11 +720,14 @@ class DAGScheduler( null } - // must be run listener before possible NotSerializableException - // should be "StageSubmitted" first and then "JobEnded" - listenerBus.post(SparkListenerStageSubmitted(stageToInfos(stage), properties)) - if (tasks.size > 0) { + runningStages += stage + // SparkListenerStageSubmitted should be posted before testing whether tasks are + // serializable. If tasks are not serializable, a SparkListenerStageCompleted event + // will be posted, which should always come after a corresponding SparkListenerStageSubmitted + // event. + listenerBus.post(SparkListenerStageSubmitted(stage.info, properties)) + // Preemptively serialize a task to make sure it can be serialized. We are catching this // exception here because it would be fairly hard to catch the non-serializable exception // down the road, where we have several different implementations for local scheduler and @@ -776,11 +746,11 @@ class DAGScheduler( } logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") - myPending ++= tasks - logDebug("New pending tasks: " + myPending) + stage.pendingTasks ++= tasks + logDebug("New pending tasks: " + stage.pendingTasks) taskScheduler.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) - stageToInfos(stage).submissionTime = Some(clock.getTime()) + stage.info.submissionTime = Some(clock.getTime()) } else { logDebug("Stage " + stage + " is actually done; %b %d %d".format( stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions)) @@ -805,13 +775,13 @@ class DAGScheduler( val stage = stageIdToStage(task.stageId) def markStageAsFinished(stage: Stage) = { - val serviceTime = stageToInfos(stage).submissionTime match { + val serviceTime = stage.info.submissionTime match { case Some(t) => "%.03f".format((clock.getTime() - t) / 1000.0) case _ => "Unknown" } logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) - stageToInfos(stage).completionTime = Some(clock.getTime()) - listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) + stage.info.completionTime = Some(clock.getTime()) + listenerBus.post(SparkListenerStageCompleted(stage.info)) runningStages -= stage } event.reason match { @@ -820,10 +790,10 @@ class DAGScheduler( // TODO: fail the stage if the accumulator update fails... Accumulators.add(event.accumUpdates) // TODO: do this only if task wasn't resubmitted } - pendingTasks(stage) -= task + stage.pendingTasks -= task task match { case rt: ResultTask[_, _] => - resultStageToJob.get(stage) match { + stage.resultOfJob match { case Some(job) => if (!job.finished(rt.outputId)) { job.finished(rt.outputId) = true @@ -831,7 +801,7 @@ class DAGScheduler( // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { markStageAsFinished(stage) - cleanupStateForJobAndIndependentStages(job, Some(stage)) + cleanupStateForJobAndIndependentStages(job) listenerBus.post(SparkListenerJobEnd(job.jobId, JobSucceeded)) } @@ -858,7 +828,7 @@ class DAGScheduler( } else { stage.addOutputLoc(smt.partitionId, status) } - if (runningStages.contains(stage) && pendingTasks(stage).isEmpty) { + if (runningStages.contains(stage) && stage.pendingTasks.isEmpty) { markStageAsFinished(stage) logInfo("looking for newly runnable stages") logInfo("running: " + runningStages) @@ -907,7 +877,7 @@ class DAGScheduler( case Resubmitted => logInfo("Resubmitted " + task + ", so marking it as still running") - pendingTasks(stage) += task + stage.pendingTasks += task case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => // Mark the stage that the reducer was in as unrunnable @@ -992,13 +962,14 @@ class DAGScheduler( } private[scheduler] def handleStageCancellation(stageId: Int) { - if (stageIdToJobIds.contains(stageId)) { - val jobsThatUseStage: Array[Int] = stageIdToJobIds(stageId).toArray - jobsThatUseStage.foreach(jobId => { - handleJobCancellation(jobId, "because Stage %s was cancelled".format(stageId)) - }) - } else { - logInfo("No active jobs to kill for Stage " + stageId) + stageIdToStage.get(stageId) match { + case Some(stage) => + val jobsThatUseStage: Array[Int] = stage.jobIds.toArray + jobsThatUseStage.foreach { jobId => + handleJobCancellation(jobId, s"because Stage $stageId was cancelled") + } + case None => + logInfo("No active jobs to kill for Stage " + stageId) } submitWaitingStages() } @@ -1007,8 +978,8 @@ class DAGScheduler( if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { - failJobAndIndependentStages(jobIdToActiveJob(jobId), - "Job %d cancelled %s".format(jobId, reason), None) + failJobAndIndependentStages( + jobIdToActiveJob(jobId), "Job %d cancelled %s".format(jobId, reason)) } submitWaitingStages() } @@ -1022,26 +993,21 @@ class DAGScheduler( // Skip all the actions if the stage has been removed. return } - val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq - stageToInfos(failedStage).completionTime = Some(clock.getTime()) - for (resultStage <- dependentStages) { - val job = resultStageToJob(resultStage) - failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason", - Some(resultStage)) + val dependentJobs: Seq[ActiveJob] = + activeJobs.filter(job => stageDependsOn(job.finalStage, failedStage)).toSeq + failedStage.info.completionTime = Some(clock.getTime()) + for (job <- dependentJobs) { + failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason") } - if (dependentStages.isEmpty) { + if (dependentJobs.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") } } /** * Fails a job and all stages that are only used by that job, and cleans up relevant state. - * - * @param resultStage The result stage for the job, if known. Used to cleanup state for the job - * slightly more efficiently than when not specified. */ - private def failJobAndIndependentStages(job: ActiveJob, failureReason: String, - resultStage: Option[Stage]) { + private def failJobAndIndependentStages(job: ActiveJob, failureReason: String) { val error = new SparkException(failureReason) var ableToCancelStages = true @@ -1055,7 +1021,7 @@ class DAGScheduler( logError("No stages registered for job " + job.jobId) } stages.foreach { stageId => - val jobsForStage = stageIdToJobIds.get(stageId) + val jobsForStage: Option[HashSet[Int]] = stageIdToStage.get(stageId).map(_.jobIds) if (jobsForStage.isEmpty || !jobsForStage.get.contains(job.jobId)) { logError( "Job %d not registered for stage %d even though that stage was registered for the job" @@ -1069,9 +1035,8 @@ class DAGScheduler( if (runningStages.contains(stage)) { try { // cancelTasks will fail if a SchedulerBackend does not implement killTask taskScheduler.cancelTasks(stageId, shouldInterruptThread) - val stageInfo = stageToInfos(stage) - stageInfo.stageFailed(failureReason) - listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) + stage.info.stageFailed(failureReason) + listenerBus.post(SparkListenerStageCompleted(stage.info)) } catch { case e: UnsupportedOperationException => logInfo(s"Could not cancel tasks for stage $stageId", e) @@ -1084,7 +1049,7 @@ class DAGScheduler( if (ableToCancelStages) { job.listener.jobFailed(error) - cleanupStateForJobAndIndependentStages(job, resultStage) + cleanupStateForJobAndIndependentStages(job) listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 798cbc598d36e..800905413d145 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import scala.collection.mutable.HashSet + import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.BlockManagerId @@ -56,8 +58,22 @@ private[spark] class Stage( val numPartitions = rdd.partitions.size val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil) var numAvailableOutputs = 0 + + /** Set of jobs that this stage belongs to. */ + val jobIds = new HashSet[Int] + + /** For stages that are the final (consists of only ResultTasks), link to the ActiveJob. */ + var resultOfJob: Option[ActiveJob] = None + var pendingTasks = new HashSet[Task[_]] + private var nextAttemptId = 0 + val name = callSite.shortForm + val details = callSite.longForm + + /** Pointer to the [StageInfo] object, set by DAGScheduler. */ + var info: StageInfo = StageInfo.fromStage(this) + def isAvailable: Boolean = { if (!isShuffleMap) { true @@ -108,9 +124,6 @@ private[spark] class Stage( def attemptId: Int = nextAttemptId - val name = callSite.shortForm - val details = callSite.longForm - override def toString = "Stage " + id override def hashCode(): Int = id diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 0a7e1ec539679..a7fa057ee05f7 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -108,7 +108,7 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize */ @DeveloperApi class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { - private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 10000) + private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 100) def newInstance(): SerializerInstance = new JavaSerializerInstance(counterReset) diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala index d45258c0a492b..76cdb8f4f8e8a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -18,6 +18,7 @@ package org.apache.spark.shuffle.hash import org.apache.spark.{InterruptibleIterator, TaskContext} +import org.apache.spark.rdd.SortOrder import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleReader} @@ -38,7 +39,7 @@ class HashShuffleReader[K, C]( val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, Serializer.getSerializer(dep.serializer)) - if (dep.aggregator.isDefined) { + val aggregatedIter: Iterator[Product2[K, C]] = if (dep.aggregator.isDefined) { if (dep.mapSideCombine) { new InterruptibleIterator(context, dep.aggregator.get.combineCombinersByKey(iter, context)) } else { @@ -49,6 +50,17 @@ class HashShuffleReader[K, C]( } else { iter } + + val sortedIter = for (sortOrder <- dep.sortOrder; ordering <- dep.keyOrdering) yield { + val buf = aggregatedIter.toArray + if (sortOrder == SortOrder.ASCENDING) { + buf.sortWith((x, y) => ordering.lt(x._1, y._1)).iterator + } else { + buf.sortWith((x, y) => ordering.gt(x._1, y._1)).iterator + } + } + + sortedIter.getOrElse(aggregatedIter) } /** Close this reader */ 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 0db0a5bc7341b..d746526639e58 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -38,7 +38,7 @@ import org.apache.spark.util._ private[spark] sealed trait BlockValues private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues private[spark] case class IteratorValues(iterator: Iterator[Any]) extends BlockValues -private[spark] case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends BlockValues +private[spark] case class ArrayValues(buffer: Array[Any]) extends BlockValues /* Class for returning a fetched block and associated metrics. */ private[spark] class BlockResult( @@ -71,9 +71,9 @@ private[spark] class BlockManager( // Actual storage of where blocks are kept private var tachyonInitialized = false - private[storage] val memoryStore = new MemoryStore(this, maxMemory) - private[storage] val diskStore = new DiskStore(this, diskBlockManager) - private[storage] lazy val tachyonStore: TachyonStore = { + private[spark] val memoryStore = new MemoryStore(this, maxMemory) + private[spark] val diskStore = new DiskStore(this, diskBlockManager) + private[spark] 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}" @@ -463,16 +463,17 @@ private[spark] class BlockManager( val values = dataDeserialize(blockId, bytes) if (level.deserialized) { // Cache the values before returning them - // TODO: Consider creating a putValues that also takes in a iterator? - val valuesBuffer = new ArrayBuffer[Any] - valuesBuffer ++= values - memoryStore.putValues(blockId, valuesBuffer, level, returnValues = true).data - match { - case Left(values2) => - return Some(new BlockResult(values2, DataReadMethod.Disk, info.size)) - case _ => - throw new SparkException("Memory store did not return back an iterator") - } + val putResult = memoryStore.putIterator( + blockId, values, level, returnValues = true, allowPersistToDisk = false) + // The put may or may not have succeeded, depending on whether there was enough + // space to unroll the block. Either way, the put here should return an iterator. + putResult.data match { + case Left(it) => + return Some(new BlockResult(it, DataReadMethod.Disk, info.size)) + case _ => + // This only happens if we dropped the values back to disk (which is never) + throw new SparkException("Memory store did not return an iterator!") + } } else { return Some(new BlockResult(values, DataReadMethod.Disk, info.size)) } @@ -561,13 +562,14 @@ private[spark] class BlockManager( iter } - def put( + def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, - tellMaster: Boolean): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true, + effectiveStorageLevel: Option[StorageLevel] = None): Seq[(BlockId, BlockStatus)] = { require(values != null, "Values is null") - doPut(blockId, IteratorValues(values), level, tellMaster) + doPut(blockId, IteratorValues(values), level, tellMaster, effectiveStorageLevel) } /** @@ -589,13 +591,14 @@ private[spark] class BlockManager( * Put a new block of values to the block manager. * Return a list of blocks updated as a result of this put. */ - def put( + def putArray( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, - tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true, + effectiveStorageLevel: Option[StorageLevel] = None): Seq[(BlockId, BlockStatus)] = { require(values != null, "Values is null") - doPut(blockId, ArrayBufferValues(values), level, tellMaster) + doPut(blockId, ArrayValues(values), level, tellMaster, effectiveStorageLevel) } /** @@ -606,19 +609,33 @@ private[spark] class BlockManager( blockId: BlockId, bytes: ByteBuffer, level: StorageLevel, - tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true, + effectiveStorageLevel: Option[StorageLevel] = None): Seq[(BlockId, BlockStatus)] = { require(bytes != null, "Bytes is null") - doPut(blockId, ByteBufferValues(bytes), level, tellMaster) + doPut(blockId, ByteBufferValues(bytes), level, tellMaster, effectiveStorageLevel) } + /** + * Put the given block according to the given level in one of the block stores, replicating + * the values if necessary. + * + * The effective storage level refers to the level according to which the block will actually be + * handled. This allows the caller to specify an alternate behavior of doPut while preserving + * the original level specified by the user. + */ private def doPut( blockId: BlockId, data: BlockValues, level: StorageLevel, - tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true, + effectiveStorageLevel: Option[StorageLevel] = None) + : Seq[(BlockId, BlockStatus)] = { require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") + effectiveStorageLevel.foreach { level => + require(level != null && level.isValid, "Effective StorageLevel is null or invalid") + } // Return value val updatedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] @@ -657,13 +674,16 @@ private[spark] class BlockManager( // Size of the block in bytes var size = 0L + // The level we actually use to put the block + val putLevel = effectiveStorageLevel.getOrElse(level) + // If we're storing bytes, then initiate the replication before storing them locally. // This is faster as data is already serialized and ready to send. val replicationFuture = data match { - case b: ByteBufferValues if level.replication > 1 => + case b: ByteBufferValues if putLevel.replication > 1 => // Duplicate doesn't copy the bytes, but just creates a wrapper val bufferView = b.buffer.duplicate() - Future { replicate(blockId, bufferView, level) } + Future { replicate(blockId, bufferView, putLevel) } case _ => null } @@ -676,18 +696,18 @@ private[spark] class BlockManager( // returnValues - Whether to return the values put // blockStore - The type of storage to put these values into val (returnValues, blockStore: BlockStore) = { - if (level.useMemory) { + if (putLevel.useMemory) { // Put it in memory first, even if it also has useDisk set to true; // We will drop it to disk later if the memory store can't hold it. (true, memoryStore) - } else if (level.useOffHeap) { + } else if (putLevel.useOffHeap) { // Use tachyon for off-heap storage (false, tachyonStore) - } else if (level.useDisk) { + } else if (putLevel.useDisk) { // Don't get back the bytes from put unless we replicate them - (level.replication > 1, diskStore) + (putLevel.replication > 1, diskStore) } else { - assert(level == StorageLevel.NONE) + assert(putLevel == StorageLevel.NONE) throw new BlockException( blockId, s"Attempted to put block $blockId without specifying storage level!") } @@ -696,22 +716,22 @@ private[spark] class BlockManager( // Actually put the values val result = data match { case IteratorValues(iterator) => - blockStore.putValues(blockId, iterator, level, returnValues) - case ArrayBufferValues(array) => - blockStore.putValues(blockId, array, level, returnValues) + blockStore.putIterator(blockId, iterator, putLevel, returnValues) + case ArrayValues(array) => + blockStore.putArray(blockId, array, putLevel, returnValues) case ByteBufferValues(bytes) => bytes.rewind() - blockStore.putBytes(blockId, bytes, level) + blockStore.putBytes(blockId, bytes, putLevel) } size = result.size result.data match { - case Left (newIterator) if level.useMemory => valuesAfterPut = newIterator + case Left (newIterator) if putLevel.useMemory => valuesAfterPut = newIterator case Right (newBytes) => bytesAfterPut = newBytes case _ => } // Keep track of which blocks are dropped from memory - if (level.useMemory) { + if (putLevel.useMemory) { result.droppedBlocks.foreach { updatedBlocks += _ } } @@ -742,7 +762,7 @@ private[spark] class BlockManager( // Either we're storing bytes and we asynchronously started replication, or we're storing // values and need to serialize and replicate them now: - if (level.replication > 1) { + if (putLevel.replication > 1) { data match { case ByteBufferValues(bytes) => if (replicationFuture != null) { @@ -758,7 +778,7 @@ private[spark] class BlockManager( } bytesAfterPut = dataSerialize(blockId, valuesAfterPut) } - replicate(blockId, bytesAfterPut, level) + replicate(blockId, bytesAfterPut, putLevel) logDebug("Put block %s remotely took %s" .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) } @@ -766,7 +786,7 @@ private[spark] class BlockManager( BlockManager.dispose(bytesAfterPut) - if (level.replication > 1) { + if (putLevel.replication > 1) { logDebug("Putting block %s with replication took %s" .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } else { @@ -818,7 +838,7 @@ private[spark] class BlockManager( value: Any, level: StorageLevel, tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { - put(blockId, Iterator(value), level, tellMaster) + putIterator(blockId, Iterator(value), level, tellMaster) } /** @@ -829,7 +849,7 @@ private[spark] class BlockManager( */ def dropFromMemory( blockId: BlockId, - data: Either[ArrayBuffer[Any], ByteBuffer]): Option[BlockStatus] = { + data: Either[Array[Any], ByteBuffer]): Option[BlockStatus] = { logInfo(s"Dropping block $blockId from memory") val info = blockInfo.get(blockId).orNull @@ -853,7 +873,7 @@ private[spark] class BlockManager( logInfo(s"Writing block $blockId to disk") data match { case Left(elements) => - diskStore.putValues(blockId, elements, level, returnValues = false) + diskStore.putArray(blockId, elements, level, returnValues = false) case Right(bytes) => diskStore.putBytes(blockId, bytes, level) } @@ -1068,9 +1088,11 @@ private[spark] class BlockManager( private[spark] object BlockManager extends Logging { private val ID_GENERATOR = new IdGenerator + /** Return the total amount of storage memory available. */ private def getMaxMemory(conf: SparkConf): Long = { val memoryFraction = conf.getDouble("spark.storage.memoryFraction", 0.6) - (Runtime.getRuntime.maxMemory * memoryFraction).toLong + val safetyFraction = conf.getDouble("spark.storage.safetyFraction", 0.9) + (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong } def getHeartBeatFrequency(conf: SparkConf): Long = diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala index b9b53b1a2f118..69985c9759e2d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala @@ -37,15 +37,15 @@ private[spark] abstract class BlockStore(val blockManager: BlockManager) extends * @return a PutResult that contains the size of the data, as well as the values put if * returnValues is true (if not, the result's data field can be null) */ - def putValues( + def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, returnValues: Boolean): PutResult - def putValues( + def putArray( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index ebff0cb5ba153..c83261dd91b36 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -21,8 +21,6 @@ import java.io.{FileOutputStream, RandomAccessFile} import java.nio.ByteBuffer import java.nio.channels.FileChannel.MapMode -import scala.collection.mutable.ArrayBuffer - import org.apache.spark.Logging import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils @@ -30,7 +28,7 @@ import org.apache.spark.util.Utils /** * Stores BlockManager blocks on disk. */ -private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) +private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) extends BlockStore(blockManager) with Logging { val minMemoryMapBytes = blockManager.conf.getLong("spark.storage.memoryMapThreshold", 2 * 4096L) @@ -57,15 +55,15 @@ private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManage PutResult(bytes.limit(), Right(bytes.duplicate())) } - override def putValues( + override def putArray( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - putValues(blockId, values.toIterator, level, returnValues) + putIterator(blockId, values.toIterator, level, returnValues) } - override def putValues( + override def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 71f66c826c5b3..28f675c2bbb1e 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -20,27 +20,45 @@ package org.apache.spark.storage import java.nio.ByteBuffer import java.util.LinkedHashMap +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark.util.{SizeEstimator, Utils} +import org.apache.spark.util.collection.SizeTrackingVector private case class MemoryEntry(value: Any, size: Long, deserialized: Boolean) /** - * Stores blocks in memory, either as ArrayBuffers of deserialized Java objects or as + * Stores blocks in memory, either as Arrays of deserialized Java objects or as * serialized ByteBuffers. */ -private class MemoryStore(blockManager: BlockManager, maxMemory: Long) +private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) extends BlockStore(blockManager) { + private val conf = blockManager.conf private val entries = new LinkedHashMap[BlockId, MemoryEntry](32, 0.75f, true) + @volatile private var currentMemory = 0L - // Object used to ensure that only one thread is putting blocks and if necessary, dropping - // blocks from the memory store. - private val putLock = new Object() + + // Ensure only one thread is putting, and if necessary, dropping blocks at any given time + private val accountingLock = new Object + + // A mapping from thread ID to amount of memory used for unrolling a block (in bytes) + // All accesses of this map are assumed to have manually synchronized on `accountingLock` + private val unrollMemoryMap = mutable.HashMap[Long, Long]() + + /** + * The amount of space ensured for unrolling values in memory, shared across all cores. + * This space is not reserved in advance, but allocated dynamically by dropping existing blocks. + */ + private val maxUnrollMemory: Long = { + val unrollFraction = conf.getDouble("spark.storage.unrollFraction", 0.2) + (maxMemory * unrollFraction).toLong + } logInfo("MemoryStore started with capacity %s".format(Utils.bytesToString(maxMemory))) + /** Free memory not occupied by existing blocks. Note that this does not include unroll memory. */ def freeMemory: Long = maxMemory - currentMemory override def getSize(blockId: BlockId): Long = { @@ -55,20 +73,16 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) bytes.rewind() if (level.deserialized) { val values = blockManager.dataDeserialize(blockId, bytes) - val elements = new ArrayBuffer[Any] - elements ++= values - val sizeEstimate = SizeEstimator.estimate(elements.asInstanceOf[AnyRef]) - val putAttempt = tryToPut(blockId, elements, sizeEstimate, deserialized = true) - PutResult(sizeEstimate, Left(values.toIterator), putAttempt.droppedBlocks) + putIterator(blockId, values, level, returnValues = true) } else { val putAttempt = tryToPut(blockId, bytes, bytes.limit, deserialized = false) PutResult(bytes.limit(), Right(bytes.duplicate()), putAttempt.droppedBlocks) } } - override def putValues( + override def putArray( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult = { if (level.deserialized) { @@ -82,14 +96,52 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } - override def putValues( + override def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - val valueEntries = new ArrayBuffer[Any]() - valueEntries ++= values - putValues(blockId, valueEntries, level, returnValues) + putIterator(blockId, values, level, returnValues, allowPersistToDisk = true) + } + + /** + * Attempt to put the given block in memory store. + * + * There may not be enough space to fully unroll the iterator in memory, in which case we + * optionally drop the values to disk if + * (1) the block's storage level specifies useDisk, and + * (2) `allowPersistToDisk` is true. + * + * One scenario in which `allowPersistToDisk` is false is when the BlockManager reads a block + * back from disk and attempts to cache it in memory. In this case, we should not persist the + * block back on disk again, as it is already in disk store. + */ + private[storage] def putIterator( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + returnValues: Boolean, + allowPersistToDisk: Boolean): PutResult = { + val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + val unrolledValues = unrollSafely(blockId, values, droppedBlocks) + unrolledValues match { + case Left(arrayValues) => + // Values are fully unrolled in memory, so store them as an array + val res = putArray(blockId, arrayValues, level, returnValues) + droppedBlocks ++= res.droppedBlocks + PutResult(res.size, res.data, droppedBlocks) + case Right(iteratorValues) => + // Not enough space to unroll this block; drop to disk if applicable + logWarning(s"Not enough space to store block $blockId in memory! " + + s"Free memory is $freeMemory bytes.") + if (level.useDisk && allowPersistToDisk) { + logWarning(s"Persisting block $blockId to disk instead.") + val res = blockManager.diskStore.putIterator(blockId, iteratorValues, level, returnValues) + PutResult(res.size, res.data, droppedBlocks) + } else { + PutResult(0, Left(iteratorValues), droppedBlocks) + } + } } override def getBytes(blockId: BlockId): Option[ByteBuffer] = { @@ -99,7 +151,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) if (entry == null) { None } else if (entry.deserialized) { - Some(blockManager.dataSerialize(blockId, entry.value.asInstanceOf[ArrayBuffer[Any]].iterator)) + Some(blockManager.dataSerialize(blockId, entry.value.asInstanceOf[Array[Any]].iterator)) } else { Some(entry.value.asInstanceOf[ByteBuffer].duplicate()) // Doesn't actually copy the data } @@ -112,7 +164,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) if (entry == null) { None } else if (entry.deserialized) { - Some(entry.value.asInstanceOf[ArrayBuffer[Any]].iterator) + Some(entry.value.asInstanceOf[Array[Any]].iterator) } else { val buffer = entry.value.asInstanceOf[ByteBuffer].duplicate() // Doesn't actually copy data Some(blockManager.dataDeserialize(blockId, buffer)) @@ -140,6 +192,93 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) logInfo("MemoryStore cleared") } + /** + * Unroll the given block in memory safely. + * + * The safety of this operation refers to avoiding potential OOM exceptions caused by + * unrolling the entirety of the block in memory at once. This is achieved by periodically + * checking whether the memory restrictions for unrolling blocks are still satisfied, + * stopping immediately if not. This check is a safeguard against the scenario in which + * there is not enough free memory to accommodate the entirety of a single block. + * + * This method returns either an array with the contents of the entire block or an iterator + * containing the values of the block (if the array would have exceeded available memory). + */ + def unrollSafely( + blockId: BlockId, + values: Iterator[Any], + droppedBlocks: ArrayBuffer[(BlockId, BlockStatus)]) + : Either[Array[Any], Iterator[Any]] = { + + // Number of elements unrolled so far + var elementsUnrolled = 0 + // Whether there is still enough memory for us to continue unrolling this block + var keepUnrolling = true + // Initial per-thread memory to request for unrolling blocks (bytes). Exposed for testing. + val initialMemoryThreshold = conf.getLong("spark.storage.unrollMemoryThreshold", 1024 * 1024) + // How often to check whether we need to request more memory + val memoryCheckPeriod = 16 + // Memory currently reserved by this thread for this particular unrolling operation + var memoryThreshold = initialMemoryThreshold + // Memory to request as a multiple of current vector size + val memoryGrowthFactor = 1.5 + // Previous unroll memory held by this thread, for releasing later (only at the very end) + val previousMemoryReserved = currentUnrollMemoryForThisThread + // Underlying vector for unrolling the block + var vector = new SizeTrackingVector[Any] + + // Request enough memory to begin unrolling + keepUnrolling = reserveUnrollMemoryForThisThread(initialMemoryThreshold) + + // Unroll this block safely, checking whether we have exceeded our threshold periodically + try { + while (values.hasNext && keepUnrolling) { + vector += values.next() + if (elementsUnrolled % memoryCheckPeriod == 0) { + // If our vector's size has exceeded the threshold, request more memory + val currentSize = vector.estimateSize() + if (currentSize >= memoryThreshold) { + val amountToRequest = (currentSize * (memoryGrowthFactor - 1)).toLong + // Hold the accounting lock, in case another thread concurrently puts a block that + // takes up the unrolling space we just ensured here + accountingLock.synchronized { + if (!reserveUnrollMemoryForThisThread(amountToRequest)) { + // If the first request is not granted, try again after ensuring free space + // If there is still not enough space, give up and drop the partition + val spaceToEnsure = maxUnrollMemory - currentUnrollMemory + if (spaceToEnsure > 0) { + val result = ensureFreeSpace(blockId, spaceToEnsure) + droppedBlocks ++= result.droppedBlocks + } + keepUnrolling = reserveUnrollMemoryForThisThread(amountToRequest) + } + } + // New threshold is currentSize * memoryGrowthFactor + memoryThreshold = currentSize + amountToRequest + } + } + elementsUnrolled += 1 + } + + if (keepUnrolling) { + // We successfully unrolled the entirety of this block + Left(vector.toArray) + } else { + // We ran out of space while unrolling the values for this block + Right(vector.iterator ++ values) + } + + } finally { + // If we return an array, the values returned do not depend on the underlying vector and + // we can immediately free up space for other threads. Otherwise, if we return an iterator, + // we release the memory claimed by this thread later on when the task finishes. + if (keepUnrolling) { + val amountToRelease = currentUnrollMemoryForThisThread - previousMemoryReserved + releaseUnrollMemoryForThisThread(amountToRelease) + } + } + } + /** * Return the RDD ID that a given block ID is from, or None if it is not an RDD block. */ @@ -149,10 +288,10 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) /** * Try to put in a set of values, if we can free up enough space. The value should either be - * an ArrayBuffer if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) - * size must also be passed by the caller. + * an Array if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) size + * must also be passed by the caller. * - * Lock on the object putLock to ensure that all the put requests and its associated block + * Synchronize on `accountingLock` to ensure that all the put requests and its associated block * dropping is done by only on thread at a time. Otherwise while one thread is dropping * blocks to free memory for one block, another thread may use up the freed space for * another block. @@ -174,7 +313,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) var putSuccess = false val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - putLock.synchronized { + accountingLock.synchronized { val freeSpaceResult = ensureFreeSpace(blockId, size) val enoughFreeSpace = freeSpaceResult.success droppedBlocks ++= freeSpaceResult.droppedBlocks @@ -193,7 +332,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) // Tell the block manager that we couldn't put it in memory so that it can drop it to // disk if the block allows disk storage. val data = if (deserialized) { - Left(value.asInstanceOf[ArrayBuffer[Any]]) + Left(value.asInstanceOf[Array[Any]]) } else { Right(value.asInstanceOf[ByteBuffer].duplicate()) } @@ -210,12 +349,14 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) * from the same RDD (which leads to a wasteful cyclic replacement pattern for RDDs that * don't fit into memory that we want to avoid). * - * Assume that a lock is held by the caller to ensure only one thread is dropping blocks. - * Otherwise, the freed space may fill up before the caller puts in their new value. + * Assume that `accountingLock` is held by the caller to ensure only one thread is dropping + * blocks. Otherwise, the freed space may fill up before the caller puts in their new value. * * Return whether there is enough free space, along with the blocks dropped in the process. */ - private def ensureFreeSpace(blockIdToAdd: BlockId, space: Long): ResultWithDroppedBlocks = { + private def ensureFreeSpace( + blockIdToAdd: BlockId, + space: Long): ResultWithDroppedBlocks = { logInfo(s"ensureFreeSpace($space) called with curMem=$currentMemory, maxMem=$maxMemory") val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] @@ -225,9 +366,12 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) return ResultWithDroppedBlocks(success = false, droppedBlocks) } - if (maxMemory - currentMemory < space) { + // Take into account the amount of memory currently occupied by unrolling blocks + val actualFreeMemory = freeMemory - currentUnrollMemory + + if (actualFreeMemory < space) { val rddToAdd = getRddId(blockIdToAdd) - val selectedBlocks = new ArrayBuffer[BlockId]() + val selectedBlocks = new ArrayBuffer[BlockId] var selectedMemory = 0L // This is synchronized to ensure that the set of entries is not changed @@ -235,7 +379,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) // can lead to exceptions. entries.synchronized { val iterator = entries.entrySet().iterator() - while (maxMemory - (currentMemory - selectedMemory) < space && iterator.hasNext) { + while (actualFreeMemory + selectedMemory < space && iterator.hasNext) { val pair = iterator.next() val blockId = pair.getKey if (rddToAdd.isEmpty || rddToAdd != getRddId(blockId)) { @@ -245,7 +389,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } - if (maxMemory - (currentMemory - selectedMemory) >= space) { + if (actualFreeMemory + selectedMemory >= space) { logInfo(s"${selectedBlocks.size} blocks selected for dropping") for (blockId <- selectedBlocks) { val entry = entries.synchronized { entries.get(blockId) } @@ -254,7 +398,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) // future safety. if (entry != null) { val data = if (entry.deserialized) { - Left(entry.value.asInstanceOf[ArrayBuffer[Any]]) + Left(entry.value.asInstanceOf[Array[Any]]) } else { Right(entry.value.asInstanceOf[ByteBuffer].duplicate()) } @@ -275,8 +419,56 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) override def contains(blockId: BlockId): Boolean = { entries.synchronized { entries.containsKey(blockId) } } + + /** + * Reserve additional memory for unrolling blocks used by this thread. + * Return whether the request is granted. + */ + private[spark] def reserveUnrollMemoryForThisThread(memory: Long): Boolean = { + accountingLock.synchronized { + val granted = freeMemory > currentUnrollMemory + memory + if (granted) { + val threadId = Thread.currentThread().getId + unrollMemoryMap(threadId) = unrollMemoryMap.getOrElse(threadId, 0L) + memory + } + granted + } + } + + /** + * Release memory used by this thread for unrolling blocks. + * If the amount is not specified, remove the current thread's allocation altogether. + */ + private[spark] def releaseUnrollMemoryForThisThread(memory: Long = -1L): Unit = { + val threadId = Thread.currentThread().getId + accountingLock.synchronized { + if (memory < 0) { + unrollMemoryMap.remove(threadId) + } else { + unrollMemoryMap(threadId) = unrollMemoryMap.getOrElse(threadId, memory) - memory + // If this thread claims no more unroll memory, release it completely + if (unrollMemoryMap(threadId) <= 0) { + unrollMemoryMap.remove(threadId) + } + } + } + } + + /** + * Return the amount of memory currently occupied for unrolling blocks across all threads. + */ + private[spark] def currentUnrollMemory: Long = accountingLock.synchronized { + unrollMemoryMap.values.sum + } + + /** + * Return the amount of memory currently occupied for unrolling blocks by this thread. + */ + private[spark] def currentUnrollMemoryForThisThread: Long = accountingLock.synchronized { + unrollMemoryMap.getOrElse(Thread.currentThread().getId, 0L) + } } -private case class ResultWithDroppedBlocks( +private[spark] case class ResultWithDroppedBlocks( success: Boolean, droppedBlocks: Seq[(BlockId, BlockStatus)]) diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index d8ff4ff6bd42c..932b5616043b4 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -20,8 +20,6 @@ package org.apache.spark.storage import java.io.IOException import java.nio.ByteBuffer -import scala.collection.mutable.ArrayBuffer - import tachyon.client.{ReadType, WriteType} import org.apache.spark.Logging @@ -30,7 +28,7 @@ import org.apache.spark.util.Utils /** * Stores BlockManager blocks on Tachyon. */ -private class TachyonStore( +private[spark] class TachyonStore( blockManager: BlockManager, tachyonManager: TachyonBlockManager) extends BlockStore(blockManager: BlockManager) with Logging { @@ -45,15 +43,15 @@ private class TachyonStore( putIntoTachyonStore(blockId, bytes, returnValues = true) } - override def putValues( + override def putArray( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - putValues(blockId, values.toIterator, level, returnValues) + putIterator(blockId, values.toIterator, level, returnValues) } - override def putValues( + override def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index 328be158db680..75c2e09a6bbb8 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -48,7 +48,7 @@ private[spark] object ThreadingTest { val block = (1 to blockSize).map(_ => Random.nextInt()) val level = randomLevel() val startTime = System.currentTimeMillis() - manager.put(blockId, block.iterator, level, tellMaster = true) + manager.putIterator(blockId, block.iterator, level, tellMaster = true) println("Pushed block " + blockId + " in " + (System.currentTimeMillis - startTime) + " ms") queue.add((blockId, block)) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index f8b308c981548..3dcfaf76e4aba 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -119,14 +119,14 @@ private[ui] class StageTableBase( } - val stageDataOption = listener.stageIdToData.get(s.stageId) - // Too many nested map/flatMaps with options are just annoying to read. Do this imperatively. - if (stageDataOption.isDefined && stageDataOption.get.description.isDefined) { - val desc = stageDataOption.get.description -
{desc}
{killLink} {nameLink} {details}
- } else { -
{killLink} {nameLink} {details}
+ val stageDesc = for { + stageData <- listener.stageIdToData.get(s.stageId) + desc <- stageData.description + } yield { +
{desc}
} + +
{stageDesc.getOrElse("")} {killLink} {nameLink} {details}
} protected def stageRow(s: StageInfo): Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index 08465575309c6..bce3b3afe9aba 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -180,7 +180,7 @@ private[spark] object SizeEstimator extends Logging { } } - // Estimat the size of arrays larger than ARRAY_SIZE_FOR_SAMPLING by sampling. + // Estimate the size of arrays larger than ARRAY_SIZE_FOR_SAMPLING by sampling. private val ARRAY_SIZE_FOR_SAMPLING = 200 private val ARRAY_SAMPLE_SIZE = 100 // should be lower than ARRAY_SIZE_FOR_SAMPLING 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 1a4f4eba98ea8..8cbb9050f393b 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -21,7 +21,7 @@ import java.io._ import java.net.{InetAddress, Inet4Address, NetworkInterface, URI, URL, URLConnection} import java.nio.ByteBuffer import java.util.{Locale, Random, UUID} -import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} +import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} import scala.collection.JavaConversions._ import scala.collection.Map @@ -553,19 +553,19 @@ private[spark] object Utils extends Logging { new ThreadFactoryBuilder().setDaemon(true) /** - * Wrapper over newCachedThreadPool. Thread names are formatted as prefix-ID, where ID is a - * unique, sequentially assigned integer. + * Create a thread factory that names threads with a prefix and also sets the threads to daemon. */ - def newDaemonCachedThreadPool(prefix: String): ThreadPoolExecutor = { - val threadFactory = daemonThreadFactoryBuilder.setNameFormat(prefix + "-%d").build() - Executors.newCachedThreadPool(threadFactory).asInstanceOf[ThreadPoolExecutor] + def namedThreadFactory(prefix: String): ThreadFactory = { + daemonThreadFactoryBuilder.setNameFormat(prefix + "-%d").build() } /** - * Return the string to tell how long has passed in milliseconds. + * Wrapper over newCachedThreadPool. Thread names are formatted as prefix-ID, where ID is a + * unique, sequentially assigned integer. */ - def getUsedTimeMs(startTimeMs: Long): String = { - " " + (System.currentTimeMillis - startTimeMs) + " ms" + def newDaemonCachedThreadPool(prefix: String): ThreadPoolExecutor = { + val threadFactory = namedThreadFactory(prefix) + Executors.newCachedThreadPool(threadFactory).asInstanceOf[ThreadPoolExecutor] } /** @@ -573,10 +573,17 @@ private[spark] object Utils extends Logging { * unique, sequentially assigned integer. */ def newDaemonFixedThreadPool(nThreads: Int, prefix: String): ThreadPoolExecutor = { - val threadFactory = daemonThreadFactoryBuilder.setNameFormat(prefix + "-%d").build() + val threadFactory = namedThreadFactory(prefix) Executors.newFixedThreadPool(nThreads, threadFactory).asInstanceOf[ThreadPoolExecutor] } + /** + * Return the string to tell how long has passed in milliseconds. + */ + def getUsedTimeMs(startTimeMs: Long): String = { + " " + (System.currentTimeMillis - startTimeMs) + " ms" + } + private def listFilesSafely(file: File): Seq[File] = { val files = file.listFiles() if (files == null) { diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index be8f6529f7a1c..6f263c39d1435 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -74,7 +74,7 @@ class ExternalAppendOnlyMap[K, V, C]( // Collective memory threshold shared across all running tasks private val maxMemoryThreshold = { - val memoryFraction = sparkConf.getDouble("spark.shuffle.memoryFraction", 0.3) + val memoryFraction = sparkConf.getDouble("spark.shuffle.memoryFraction", 0.2) val safetyFraction = sparkConf.getDouble("spark.shuffle.safetyFraction", 0.8) (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong } @@ -110,42 +110,69 @@ class ExternalAppendOnlyMap[K, V, C]( /** * Insert the given key and value into the map. + */ + def insert(key: K, value: V): Unit = { + insertAll(Iterator((key, value))) + } + + /** + * Insert the given iterator of keys and values into the map. * - * If the underlying map is about to grow, check if the global pool of shuffle memory has + * When the underlying map needs to grow, check if the global pool of shuffle memory has * enough room for this to happen. If so, allocate the memory required to grow the map; * otherwise, spill the in-memory map to disk. * * The shuffle memory usage of the first trackMemoryThreshold entries is not tracked. */ - def insert(key: K, value: V) { + def insertAll(entries: Iterator[Product2[K, V]]): Unit = { + // An update function for the map that we reuse across entries to avoid allocating + // a new closure each time + var curEntry: Product2[K, V] = null val update: (Boolean, C) => C = (hadVal, oldVal) => { - if (hadVal) mergeValue(oldVal, value) else createCombiner(value) + if (hadVal) mergeValue(oldVal, curEntry._2) else createCombiner(curEntry._2) } - if (numPairsInMemory > trackMemoryThreshold && currentMap.atGrowThreshold) { - val mapSize = currentMap.estimateSize() - var shouldSpill = false - val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap - - // Atomically check whether there is sufficient memory in the global pool for - // this map to grow and, if possible, allocate the required amount - shuffleMemoryMap.synchronized { - val previouslyOccupiedMemory = shuffleMemoryMap.get(threadId) - val availableMemory = maxMemoryThreshold - - (shuffleMemoryMap.values.sum - previouslyOccupiedMemory.getOrElse(0L)) - - // Assume map growth factor is 2x - shouldSpill = availableMemory < mapSize * 2 - if (!shouldSpill) { - shuffleMemoryMap(threadId) = mapSize * 2 + + while (entries.hasNext) { + curEntry = entries.next() + if (numPairsInMemory > trackMemoryThreshold && currentMap.atGrowThreshold) { + val mapSize = currentMap.estimateSize() + var shouldSpill = false + val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap + + // Atomically check whether there is sufficient memory in the global pool for + // this map to grow and, if possible, allocate the required amount + shuffleMemoryMap.synchronized { + val previouslyOccupiedMemory = shuffleMemoryMap.get(threadId) + val availableMemory = maxMemoryThreshold - + (shuffleMemoryMap.values.sum - previouslyOccupiedMemory.getOrElse(0L)) + + // Assume map growth factor is 2x + shouldSpill = availableMemory < mapSize * 2 + if (!shouldSpill) { + shuffleMemoryMap(threadId) = mapSize * 2 + } + } + // Do not synchronize spills + if (shouldSpill) { + spill(mapSize) } } - // Do not synchronize spills - if (shouldSpill) { - spill(mapSize) - } + currentMap.changeValue(curEntry._1, update) + numPairsInMemory += 1 } - currentMap.changeValue(key, update) - numPairsInMemory += 1 + } + + /** + * Insert the given iterable of keys and values into the map. + * + * When the underlying map needs to grow, check if the global pool of shuffle memory has + * enough room for this to happen. If so, allocate the memory required to grow the map; + * otherwise, spill the in-memory map to disk. + * + * The shuffle memory usage of the first trackMemoryThreshold entries is not tracked. + */ + def insertAll(entries: Iterable[Product2[K, V]]): Unit = { + insertAll(entries.iterator) } /** diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala index b84eb65c62bc7..7e76d060d6000 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala @@ -36,7 +36,7 @@ class PrimitiveVector[@specialized(Long, Int, Double) V: ClassTag](initialSize: _array(index) } - def +=(value: V) { + def +=(value: V): Unit = { if (_numElements == _array.length) { resize(_array.length * 2) } @@ -50,6 +50,19 @@ class PrimitiveVector[@specialized(Long, Int, Double) V: ClassTag](initialSize: def size: Int = _numElements + def iterator: Iterator[V] = new Iterator[V] { + var index = 0 + override def hasNext: Boolean = index < _numElements + override def next(): V = { + if (!hasNext) { + throw new NoSuchElementException + } + val value = _array(index) + index += 1 + value + } + } + /** Gets the underlying array backing this vector. */ def array: Array[V] = _array diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala new file mode 100644 index 0000000000000..3eb1010dc1e8d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.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.util.collection + +import scala.collection.mutable + +import org.apache.spark.util.SizeEstimator + +/** + * A general interface for collections to keep track of their estimated sizes in bytes. + * We sample with a slow exponential back-off using the SizeEstimator to amortize the time, + * as each call to SizeEstimator is somewhat expensive (order of a few milliseconds). + */ +private[spark] trait SizeTracker { + + import SizeTracker._ + + /** + * Controls the base of the exponential which governs the rate of sampling. + * E.g., a value of 2 would mean we sample at 1, 2, 4, 8, ... elements. + */ + private val SAMPLE_GROWTH_RATE = 1.1 + + /** Samples taken since last resetSamples(). Only the last two are kept for extrapolation. */ + private val samples = new mutable.Queue[Sample] + + /** The average number of bytes per update between our last two samples. */ + private var bytesPerUpdate: Double = _ + + /** Total number of insertions and updates into the map since the last resetSamples(). */ + private var numUpdates: Long = _ + + /** The value of 'numUpdates' at which we will take our next sample. */ + private var nextSampleNum: Long = _ + + resetSamples() + + /** + * Reset samples collected so far. + * This should be called after the collection undergoes a dramatic change in size. + */ + protected def resetSamples(): Unit = { + numUpdates = 1 + nextSampleNum = 1 + samples.clear() + takeSample() + } + + /** + * Callback to be invoked after every update. + */ + protected def afterUpdate(): Unit = { + numUpdates += 1 + if (nextSampleNum == numUpdates) { + takeSample() + } + } + + /** + * Take a new sample of the current collection's size. + */ + private def takeSample(): Unit = { + samples.enqueue(Sample(SizeEstimator.estimate(this), numUpdates)) + // Only use the last two samples to extrapolate + if (samples.size > 2) { + samples.dequeue() + } + val bytesDelta = samples.toList.reverse match { + case latest :: previous :: tail => + (latest.size - previous.size).toDouble / (latest.numUpdates - previous.numUpdates) + // If fewer than 2 samples, assume no change + case _ => 0 + } + bytesPerUpdate = math.max(0, bytesDelta) + nextSampleNum = math.ceil(numUpdates * SAMPLE_GROWTH_RATE).toLong + } + + /** + * Estimate the current size of the collection in bytes. O(1) time. + */ + def estimateSize(): Long = { + assert(samples.nonEmpty) + val extrapolatedDelta = bytesPerUpdate * (numUpdates - samples.last.numUpdates) + (samples.last.size + extrapolatedDelta).toLong + } +} + +private object SizeTracker { + case class Sample(size: Long, numUpdates: Long) +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala index 204330dad48b9..de61e1d17fe10 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala @@ -17,85 +17,24 @@ package org.apache.spark.util.collection -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.util.SizeEstimator -import org.apache.spark.util.collection.SizeTrackingAppendOnlyMap.Sample - /** - * Append-only map that keeps track of its estimated size in bytes. - * We sample with a slow exponential back-off using the SizeEstimator to amortize the time, - * as each call to SizeEstimator can take a sizable amount of time (order of a few milliseconds). + * An append-only map that keeps track of its estimated size in bytes. */ -private[spark] class SizeTrackingAppendOnlyMap[K, V] extends AppendOnlyMap[K, V] { - - /** - * Controls the base of the exponential which governs the rate of sampling. - * E.g., a value of 2 would mean we sample at 1, 2, 4, 8, ... elements. - */ - private val SAMPLE_GROWTH_RATE = 1.1 - - /** All samples taken since last resetSamples(). Only the last two are used for extrapolation. */ - private val samples = new ArrayBuffer[Sample]() - - /** Total number of insertions and updates into the map since the last resetSamples(). */ - private var numUpdates: Long = _ - - /** The value of 'numUpdates' at which we will take our next sample. */ - private var nextSampleNum: Long = _ - - /** The average number of bytes per update between our last two samples. */ - private var bytesPerUpdate: Double = _ - - resetSamples() - - /** Called after the map grows in size, as this can be a dramatic change for small objects. */ - def resetSamples() { - numUpdates = 1 - nextSampleNum = 1 - samples.clear() - takeSample() - } +private[spark] class SizeTrackingAppendOnlyMap[K, V] extends AppendOnlyMap[K, V] with SizeTracker { override def update(key: K, value: V): Unit = { super.update(key, value) - numUpdates += 1 - if (nextSampleNum == numUpdates) { takeSample() } + super.afterUpdate() } override def changeValue(key: K, updateFunc: (Boolean, V) => V): V = { val newValue = super.changeValue(key, updateFunc) - numUpdates += 1 - if (nextSampleNum == numUpdates) { takeSample() } + super.afterUpdate() newValue } - /** Takes a new sample of the current map's size. */ - def takeSample() { - samples += Sample(SizeEstimator.estimate(this), numUpdates) - // Only use the last two samples to extrapolate. If fewer than 2 samples, assume no change. - bytesPerUpdate = math.max(0, samples.toSeq.reverse match { - case latest :: previous :: tail => - (latest.size - previous.size).toDouble / (latest.numUpdates - previous.numUpdates) - case _ => - 0 - }) - nextSampleNum = math.ceil(numUpdates * SAMPLE_GROWTH_RATE).toLong - } - - override protected def growTable() { + override protected def growTable(): Unit = { super.growTable() resetSamples() } - - /** Estimates the current size of the map in bytes. O(1) time. */ - def estimateSize(): Long = { - assert(samples.nonEmpty) - val extrapolatedDelta = bytesPerUpdate * (numUpdates - samples.last.numUpdates) - (samples.last.size + extrapolatedDelta).toLong - } -} - -private object SizeTrackingAppendOnlyMap { - case class Sample(size: Long, numUpdates: Long) } diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala new file mode 100644 index 0000000000000..65a7b4e0d497b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala @@ -0,0 +1,46 @@ +/* + * 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.util.collection + +import scala.reflect.ClassTag + +/** + * An append-only buffer that keeps track of its estimated size in bytes. + */ +private[spark] class SizeTrackingVector[T: ClassTag] + extends PrimitiveVector[T] + with SizeTracker { + + override def +=(value: T): Unit = { + super.+=(value) + super.afterUpdate() + } + + override def resize(newLength: Int): PrimitiveVector[T] = { + super.resize(newLength) + resetSamples() + this + } + + /** + * Return a trimmed version of the underlying array. + */ + def toArray: Array[T] = { + super.iterator.toArray + } +} diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index b2868b59ce6c6..f882a8623fd84 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -118,8 +118,7 @@ public void intersection() { JavaRDD intersections = s1.intersection(s2); Assert.assertEquals(3, intersections.count()); - List list = new ArrayList(); - JavaRDD empty = sc.parallelize(list); + JavaRDD empty = sc.emptyRDD(); JavaRDD emptyIntersection = empty.intersection(s2); Assert.assertEquals(0, emptyIntersection.count()); @@ -184,6 +183,12 @@ public void sortByKey() { Assert.assertEquals(new Tuple2(3, 2), sortedPairs.get(2)); } + @Test + public void emptyRDD() { + JavaRDD rdd = sc.emptyRDD(); + Assert.assertEquals("Empty RDD shouldn't have any values", 0, rdd.count()); + } + @Test public void sortBy() { List> pairs = new ArrayList>(); diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index 7f5d0b061e8b0..9c5f394d3899d 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark -import scala.collection.mutable.ArrayBuffer - import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.mock.EasyMockSugar @@ -52,22 +50,21 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } test("get uncached rdd") { - expecting { - blockManager.get(RDDBlockId(0, 0)).andReturn(None) - blockManager.put(RDDBlockId(0, 0), ArrayBuffer[Any](1, 2, 3, 4), StorageLevel.MEMORY_ONLY, - true).andStubReturn(Seq[(BlockId, BlockStatus)]()) - } - - whenExecuting(blockManager) { - val context = new TaskContext(0, 0, 0) - val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) - assert(value.toList === List(1, 2, 3, 4)) - } + // Do not mock this test, because attempting to match Array[Any], which is not covariant, + // in blockManager.put is a losing battle. You have been warned. + blockManager = sc.env.blockManager + cacheManager = sc.env.cacheManager + val context = new TaskContext(0, 0, 0) + val computeValue = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) + val getValue = blockManager.get(RDDBlockId(rdd.id, split.index)) + assert(computeValue.toList === List(1, 2, 3, 4)) + assert(getValue.isDefined, "Block cached from getOrCompute is not found!") + assert(getValue.get.data.toList === List(1, 2, 3, 4)) } test("get cached rdd") { expecting { - val result = new BlockResult(ArrayBuffer(5, 6, 7).iterator, DataReadMethod.Memory, 12) + val result = new BlockResult(Array(5, 6, 7).iterator, DataReadMethod.Memory, 12) blockManager.get(RDDBlockId(0, 0)).andReturn(Some(result)) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 9f498d579a095..9021662bcf712 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -37,6 +37,29 @@ class BuggyDAGEventProcessActor extends Actor { } } +/** + * An RDD for passing to DAGScheduler. These RDDs will use the dependencies and + * preferredLocations (if any) that are passed to them. They are deliberately not executable + * so we can test that DAGScheduler does not try to execute RDDs locally. + */ +class MyRDD( + sc: SparkContext, + numPartitions: Int, + dependencies: List[Dependency[_]], + locations: Seq[Seq[String]] = Nil) extends RDD[(Int, Int)](sc, dependencies) with Serializable { + override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = + throw new RuntimeException("should not be reached") + override def getPartitions = (0 until numPartitions).map(i => new Partition { + override def index = i + }).toArray + override def getPreferredLocations(split: Partition): Seq[String] = + if (locations.isDefinedAt(split.index)) + locations(split.index) + else + Nil + override def toString: String = "DAGSchedulerSuiteRDD " + id +} + class DAGSchedulerSuiteDummyException extends Exception class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with FunSuiteLike @@ -148,34 +171,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F * Type of RDD we use for testing. Note that we should never call the real RDD compute methods. * This is a pair RDD type so it can always be used in ShuffleDependencies. */ - type MyRDD = RDD[(Int, Int)] - - /** - * Create an RDD for passing to DAGScheduler. These RDDs will use the dependencies and - * preferredLocations (if any) that are passed to them. They are deliberately not executable - * so we can test that DAGScheduler does not try to execute RDDs locally. - */ - private def makeRdd( - numPartitions: Int, - dependencies: List[Dependency[_]], - locations: Seq[Seq[String]] = Nil - ): MyRDD = { - val maxPartition = numPartitions - 1 - val newRDD = new MyRDD(sc, dependencies) { - override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = - throw new RuntimeException("should not be reached") - override def getPartitions = (0 to maxPartition).map(i => new Partition { - override def index = i - }).toArray - override def getPreferredLocations(split: Partition): Seq[String] = - if (locations.isDefinedAt(split.index)) - locations(split.index) - else - Nil - override def toString: String = "DAGSchedulerSuiteRDD " + id - } - newRDD - } + type PairOfIntsRDD = RDD[(Int, Int)] /** * Process the supplied event as if it were the top of the DAGScheduler event queue, expecting @@ -234,19 +230,19 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F override def taskSucceeded(partition: Int, value: Any) = numResults += 1 override def jobFailed(exception: Exception) = throw exception } - submit(makeRdd(0, Nil), Array(), listener = fakeListener) + submit(new MyRDD(sc, 0, Nil), Array(), listener = fakeListener) assert(numResults === 0) } test("run trivial job") { - submit(makeRdd(1, Nil), Array(0)) + submit(new MyRDD(sc, 1, Nil), Array(0)) complete(taskSets(0), List((Success, 42))) assert(results === Map(0 -> 42)) assertDataStructuresEmpty } test("local job") { - val rdd = new MyRDD(sc, Nil) { + val rdd = new PairOfIntsRDD(sc, Nil) { override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = Array(42 -> 0).iterator override def getPartitions = Array( new Partition { override def index = 0 } ) @@ -260,7 +256,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("local job oom") { - val rdd = new MyRDD(sc, Nil) { + val rdd = new PairOfIntsRDD(sc, Nil) { override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = throw new java.lang.OutOfMemoryError("test local job oom") override def getPartitions = Array( new Partition { override def index = 0 } ) @@ -274,8 +270,8 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("run trivial job w/ dependency") { - val baseRdd = makeRdd(1, Nil) - val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) + val baseRdd = new MyRDD(sc, 1, Nil) + val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) submit(finalRdd, Array(0)) complete(taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) @@ -283,8 +279,8 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("cache location preferences w/ dependency") { - val baseRdd = makeRdd(1, Nil) - val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) + val baseRdd = new MyRDD(sc, 1, Nil) + val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) cacheLocations(baseRdd.id -> 0) = Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")) submit(finalRdd, Array(0)) @@ -295,8 +291,22 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assertDataStructuresEmpty } + test("unserializable task") { + val unserializableRdd = new MyRDD(sc, 1, Nil) { + class UnserializableClass + val unserializable = new UnserializableClass + } + submit(unserializableRdd, Array(0)) + assert(failure.getMessage.startsWith( + "Job aborted due to stage failure: Task not serializable:")) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(0)) + assert(sparkListener.failedStages.size === 1) + assertDataStructuresEmpty + } + test("trivial job failure") { - submit(makeRdd(1, Nil), Array(0)) + submit(new MyRDD(sc, 1, Nil), Array(0)) failed(taskSets(0), "some failure") assert(failure.getMessage === "Job aborted due to stage failure: some failure") assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) @@ -306,7 +316,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("trivial job cancellation") { - val rdd = makeRdd(1, Nil) + val rdd = new MyRDD(sc, 1, Nil) val jobId = submit(rdd, Array(0)) cancel(jobId) assert(failure.getMessage === s"Job $jobId cancelled ") @@ -347,8 +357,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } dagEventProcessTestActor = TestActorRef[DAGSchedulerEventProcessActor]( Props(classOf[DAGSchedulerEventProcessActor], noKillScheduler))(system) - val rdd = makeRdd(1, Nil) - val jobId = submit(rdd, Array(0)) + val jobId = submit(new MyRDD(sc, 1, Nil), Array(0)) cancel(jobId) // Because the job wasn't actually cancelled, we shouldn't have received a failure message. assert(failure === null) @@ -364,10 +373,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("run trivial shuffle") { - val shuffleMapRdd = makeRdd(2, Nil) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId - val reduceRdd = makeRdd(1, List(shuffleDep)) + val reduceRdd = new MyRDD(sc, 1, List(shuffleDep)) submit(reduceRdd, Array(0)) complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), @@ -380,10 +389,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("run trivial shuffle with fetch failure") { - val shuffleMapRdd = makeRdd(2, Nil) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId - val reduceRdd = makeRdd(2, List(shuffleDep)) + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), @@ -406,10 +415,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("ignore late map task completions") { - val shuffleMapRdd = makeRdd(2, Nil) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId - val reduceRdd = makeRdd(2, List(shuffleDep)) + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) // pretend we were told hostA went away val oldEpoch = mapOutputTracker.getEpoch @@ -435,9 +444,9 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("run shuffle with map stage failure") { - val shuffleMapRdd = makeRdd(2, Nil) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) - val reduceRdd = makeRdd(2, List(shuffleDep)) + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) // Fail the map stage. This should cause the entire job to fail. @@ -472,13 +481,13 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F * without shuffleMapRdd1. */ test("failure of stage used by two jobs") { - val shuffleMapRdd1 = makeRdd(2, Nil) + val shuffleMapRdd1 = new MyRDD(sc, 2, Nil) val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, null) - val shuffleMapRdd2 = makeRdd(2, Nil) + val shuffleMapRdd2 = new MyRDD(sc, 2, Nil) val shuffleDep2 = new ShuffleDependency(shuffleMapRdd2, null) - val reduceRdd1 = makeRdd(2, List(shuffleDep1)) - val reduceRdd2 = makeRdd(2, List(shuffleDep1, shuffleDep2)) + val reduceRdd1 = new MyRDD(sc, 2, List(shuffleDep1)) + val reduceRdd2 = new MyRDD(sc, 2, List(shuffleDep1, shuffleDep2)) // We need to make our own listeners for this test, since by default submit uses the same // listener for all jobs, and here we want to capture the failure for each job separately. @@ -511,10 +520,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("run trivial shuffle with out-of-band failure and retry") { - val shuffleMapRdd = makeRdd(2, Nil) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId - val reduceRdd = makeRdd(1, List(shuffleDep)) + val reduceRdd = new MyRDD(sc, 1, List(shuffleDep)) submit(reduceRdd, Array(0)) // blockManagerMaster.removeExecutor("exec-hostA") // pretend we were told hostA went away @@ -534,11 +543,11 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("recursive shuffle failures") { - val shuffleOneRdd = makeRdd(2, Nil) + val shuffleOneRdd = new MyRDD(sc, 2, Nil) val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) - val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) + val shuffleTwoRdd = new MyRDD(sc, 2, List(shuffleDepOne)) val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) - val finalRdd = makeRdd(1, List(shuffleDepTwo)) + val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo)) submit(finalRdd, Array(0)) // have the first stage complete normally complete(taskSets(0), Seq( @@ -563,11 +572,11 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("cached post-shuffle") { - val shuffleOneRdd = makeRdd(2, Nil) + val shuffleOneRdd = new MyRDD(sc, 2, Nil) val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) - val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) + val shuffleTwoRdd = new MyRDD(sc, 2, List(shuffleDepOne)) val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) - val finalRdd = makeRdd(1, List(shuffleDepTwo)) + val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo)) submit(finalRdd, Array(0)) cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) @@ -677,15 +686,11 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F BlockManagerId("exec-" + host, host, 12345, 0) private def assertDataStructuresEmpty = { - assert(scheduler.pendingTasks.isEmpty) assert(scheduler.activeJobs.isEmpty) assert(scheduler.failedStages.isEmpty) assert(scheduler.jobIdToActiveJob.isEmpty) assert(scheduler.jobIdToStageIds.isEmpty) - assert(scheduler.stageIdToJobIds.isEmpty) assert(scheduler.stageIdToStage.isEmpty) - assert(scheduler.stageToInfos.isEmpty) - assert(scheduler.resultStageToJob.isEmpty) assert(scheduler.runningStages.isEmpty) assert(scheduler.shuffleToMapStage.isEmpty) assert(scheduler.waitingStages.isEmpty) 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 23cb6905bfdeb..dd4fd535d3577 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers import org.scalatest.time.SpanSugar._ -import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.executor.DataReadMethod import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} @@ -43,6 +43,7 @@ import scala.language.postfixOps class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter with PrivateMethodTester { + private val conf = new SparkConf(false) var store: BlockManager = null var store2: BlockManager = null @@ -61,21 +62,29 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) + private def makeBlockManager(maxMem: Long, name: String = ""): BlockManager = { + new BlockManager( + name, actorSystem, master, serializer, maxMem, conf, securityMgr, mapOutputTracker) + } + before { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0, conf = conf, - securityManager = securityMgr) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + "test", "localhost", 0, conf = conf, securityManager = securityMgr) this.actorSystem = actorSystem - conf.set("spark.driver.port", boundPort.toString) - - master = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), - conf) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case oldArch = System.setProperty("os.arch", "amd64") conf.set("os.arch", "amd64") conf.set("spark.test.useCompressedOops", "true") conf.set("spark.storage.disableBlockManagerHeartBeat", "true") + conf.set("spark.driver.port", boundPort.toString) + conf.set("spark.storage.unrollFraction", "0.4") + conf.set("spark.storage.unrollMemoryThreshold", "512") + + master = new BlockManagerMaster( + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), + conf) + val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() } @@ -138,11 +147,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("master + 1 manager interaction") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(20000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) // Putting a1, a2 and a3 in memory and telling master only about a1 and a2 store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -169,10 +177,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("master + 2 managers interaction") { - store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer(conf), 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000, "exec1") + store2 = makeBlockManager(2000, "exec2") val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -187,11 +193,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("removing block") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(20000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) // Putting a1, a2 and a3 in memory and telling master only about a1 and a2 store.putSingle("a1-to-remove", a1, StorageLevel.MEMORY_ONLY) @@ -200,8 +205,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter // Checking whether blocks are in memory and memory size val memStatus = master.getMemoryStatus.head._2 - assert(memStatus._1 == 2000L, "total memory " + memStatus._1 + " should equal 2000") - assert(memStatus._2 <= 1200L, "remaining memory " + memStatus._2 + " should <= 1200") + assert(memStatus._1 == 20000L, "total memory " + memStatus._1 + " should equal 20000") + assert(memStatus._2 <= 12000L, "remaining memory " + memStatus._2 + " should <= 12000") assert(store.getSingle("a1-to-remove").isDefined, "a1 was not in store") assert(store.getSingle("a2-to-remove").isDefined, "a2 was not in store") assert(store.getSingle("a3-to-remove").isDefined, "a3 was not in store") @@ -230,17 +235,16 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { val memStatus = master.getMemoryStatus.head._2 - memStatus._1 should equal (2000L) - memStatus._2 should equal (2000L) + memStatus._1 should equal (20000L) + memStatus._2 should equal (20000L) } } test("removing rdd") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(20000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) // Putting a1, a2 and a3 in memory. store.putSingle(rdd(0, 0), a1, StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 1), a2, StorageLevel.MEMORY_ONLY) @@ -270,11 +274,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("removing broadcast") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000) val driverStore = store - val executorStore = new BlockManager("executor", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + val executorStore = makeBlockManager(2000, "executor") val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -343,8 +345,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("reregistration on heart beat") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000) val a1 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -380,8 +381,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("reregistration doesn't dead lock") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000) val a1 = new Array[Byte](400) val a2 = List(new Array[Byte](400)) @@ -390,7 +390,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter master.removeExecutor(store.blockManagerId.executorId) val t1 = new Thread { override def run() { - store.put("a2", a2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("a2", a2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } } val t2 = new Thread { @@ -418,19 +418,14 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("correct BlockResult returned from get() calls") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr, - mapOutputTracker) - val list1 = List(new Array[Byte](200), new Array[Byte](200)) - val list1ForSizeEstimate = new ArrayBuffer[Any] - list1ForSizeEstimate ++= list1.iterator - val list1SizeEstimate = SizeEstimator.estimate(list1ForSizeEstimate) - val list2 = List(new Array[Byte](50), new Array[Byte](100), new Array[Byte](150)) - val list2ForSizeEstimate = new ArrayBuffer[Any] - list2ForSizeEstimate ++= list2.iterator - val list2SizeEstimate = SizeEstimator.estimate(list2ForSizeEstimate) - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list2memory", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list2disk", list2.iterator, StorageLevel.DISK_ONLY, tellMaster = true) + store = makeBlockManager(12000) + val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list2 = List(new Array[Byte](500), new Array[Byte](1000), new Array[Byte](1500)) + val list1SizeEstimate = SizeEstimator.estimate(list1.iterator.toArray) + val list2SizeEstimate = SizeEstimator.estimate(list2.iterator.toArray) + store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list2memory", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list2disk", list2.iterator, StorageLevel.DISK_ONLY, tellMaster = true) val list1Get = store.get("list1") assert(list1Get.isDefined, "list1 expected to be in store") assert(list1Get.get.data.size === 2) @@ -451,11 +446,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU storage") { - 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 = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY) store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY) @@ -471,11 +465,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU storage with serialization") { - 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 = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY_SER) @@ -491,11 +484,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU for partitions of same RDD") { - 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 = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle(rdd(0, 1), a1, StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 2), a2, StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 3), a3, StorageLevel.MEMORY_ONLY) @@ -511,11 +503,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU for partitions of multiple RDDs") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(0, 2), new Array[Byte](400), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(1, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store = makeBlockManager(12000) + store.putSingle(rdd(0, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(0, 2), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(1, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // At this point rdd_1_1 should've replaced rdd_0_1 assert(store.memoryStore.contains(rdd(1, 1)), "rdd_1_1 was not in store") assert(!store.memoryStore.contains(rdd(0, 1)), "rdd_0_1 was in store") @@ -523,8 +514,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter // Do a get() on rdd_0_2 so that it is the most recently used item assert(store.getSingle(rdd(0, 2)).isDefined, "rdd_0_2 was not in store") // Put in more partitions from RDD 0; they should replace rdd_1_1 - store.putSingle(rdd(0, 3), new Array[Byte](400), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(0, 4), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(0, 3), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(0, 4), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // Now rdd_1_1 should be dropped to add rdd_0_3, but then rdd_0_2 should *not* be dropped // when we try to add rdd_0_4. assert(!store.memoryStore.contains(rdd(1, 1)), "rdd_1_1 was in store") @@ -538,8 +529,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter // 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) + store = makeBlockManager(1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -555,8 +545,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("on-disk storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -569,11 +558,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage") { - 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 = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK) store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK) store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK) @@ -585,11 +573,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage with getLocalBytes") { - 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 = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK) store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK) store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK) @@ -601,11 +588,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage with serialization") { - 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 = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK_SER) store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK_SER) store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK_SER) @@ -617,11 +603,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage with serialization and getLocalBytes") { - 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 = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK_SER) store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK_SER) store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK_SER) @@ -633,12 +618,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("LRU with mixed storage levels") { - 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) - val a4 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) + val a4 = new Array[Byte](4000) // First store a1 and a2, both in memory, and a3, on disk only store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER) @@ -656,14 +640,13 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU with streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list1 = List(new Array[Byte](200), new Array[Byte](200)) - val list2 = List(new Array[Byte](200), new Array[Byte](200)) - val list3 = List(new Array[Byte](200), new Array[Byte](200)) - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list3", list3.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store = makeBlockManager(12000) + val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) + store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list2", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list3", list3.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) assert(store.get("list3").isDefined, "list3 was not in store") @@ -672,7 +655,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) // At this point list2 was gotten last, so LRU will getSingle rid of list3 - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.get("list1").isDefined, "list1 was not in store") assert(store.get("list1").get.data.size === 2) assert(store.get("list2").isDefined, "list2 was not in store") @@ -681,16 +664,15 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("LRU with mixed storage levels and streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list1 = List(new Array[Byte](200), new Array[Byte](200)) - val list2 = List(new Array[Byte](200), new Array[Byte](200)) - val list3 = List(new Array[Byte](200), new Array[Byte](200)) - val list4 = List(new Array[Byte](200), new Array[Byte](200)) + store = makeBlockManager(12000) + val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list4 = List(new Array[Byte](2000), new Array[Byte](2000)) // First store list1 and list2, both in memory, and list3, on disk only - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) - store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) - store.put("list3", list3.iterator, StorageLevel.DISK_ONLY, tellMaster = true) + store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) + store.putIterator("list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) + store.putIterator("list3", list3.iterator, StorageLevel.DISK_ONLY, tellMaster = true) val listForSizeEstimate = new ArrayBuffer[Any] listForSizeEstimate ++= list1.iterator val listSize = SizeEstimator.estimate(listForSizeEstimate) @@ -708,7 +690,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.get("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) // Now let's add in list4, which uses both disk and memory; list1 should drop out - store.put("list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) + store.putIterator("list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) assert(store.get("list1") === None, "list1 was in store") assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) @@ -731,11 +713,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("overly large block") { - store = new BlockManager("", actorSystem, master, serializer, 500, conf, - securityMgr, mapOutputTracker) - store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) + store = makeBlockManager(5000) + store.putSingle("a1", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) assert(store.getSingle("a1") === None, "a1 was in store") - store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK) + store.putSingle("a2", new Array[Byte](10000), StorageLevel.MEMORY_AND_DISK) assert(store.memoryStore.getValues("a2") === None, "a2 was in memory store") assert(store.getSingle("a2").isDefined, "a2 was not in store") } @@ -743,8 +724,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("block compression") { try { conf.set("spark.shuffle.compress", "true") - store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(20000, "exec1") store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") @@ -752,52 +732,46 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter store = null conf.set("spark.shuffle.compress", "false") - store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 1000, + store = makeBlockManager(20000, "exec2") + store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 10000, "shuffle_0_0_0 was compressed") store.stop() store = null conf.set("spark.broadcast.compress", "true") - store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 100, + store = makeBlockManager(20000, "exec3") + store.putSingle(BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 1000, "broadcast_0 was not compressed") store.stop() store = null conf.set("spark.broadcast.compress", "false") - store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 1000, "broadcast_0 was compressed") + store = makeBlockManager(20000, "exec4") + store.putSingle(BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 10000, "broadcast_0 was compressed") store.stop() store = null conf.set("spark.rdd.compress", "true") - store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(rdd(0, 0)) <= 100, "rdd_0_0 was not compressed") + store = makeBlockManager(20000, "exec5") + store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(rdd(0, 0)) <= 1000, "rdd_0_0 was not compressed") store.stop() store = null conf.set("spark.rdd.compress", "false") - store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(rdd(0, 0)) >= 1000, "rdd_0_0 was compressed") + store = makeBlockManager(20000, "exec6") + store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(rdd(0, 0)) >= 10000, "rdd_0_0 was compressed") store.stop() store = null // Check that any other block types are also kept uncompressed - store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) - assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed") + store = makeBlockManager(20000, "exec7") + store.putSingle("other_block", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) + assert(store.memoryStore.getSize("other_block") >= 10000, "other_block was compressed") store.stop() store = null } finally { @@ -871,30 +845,29 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(Arrays.equals(mappedAsArray, bytes)) assert(Arrays.equals(notMappedAsArray, bytes)) } - + test("updated block statuses") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list = List.fill(2)(new Array[Byte](200)) - val bigList = List.fill(8)(new Array[Byte](200)) + store = makeBlockManager(12000) + val list = List.fill(2)(new Array[Byte](2000)) + val bigList = List.fill(8)(new Array[Byte](2000)) // 1 updated block (i.e. list1) val updatedBlocks1 = - store.put("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks1.size === 1) assert(updatedBlocks1.head._1 === TestBlockId("list1")) assert(updatedBlocks1.head._2.storageLevel === StorageLevel.MEMORY_ONLY) // 1 updated block (i.e. list2) val updatedBlocks2 = - store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) assert(updatedBlocks2.size === 1) assert(updatedBlocks2.head._1 === TestBlockId("list2")) assert(updatedBlocks2.head._2.storageLevel === StorageLevel.MEMORY_ONLY) // 2 updated blocks - list1 is kicked out of memory while list3 is added val updatedBlocks3 = - store.put("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks3.size === 2) updatedBlocks3.foreach { case (id, status) => id match { @@ -903,11 +876,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter case _ => fail("Updated block is neither list1 nor list3") } } - assert(store.get("list3").isDefined, "list3 was not in store") + assert(store.memoryStore.contains("list3"), "list3 was not in memory store") // 2 updated blocks - list2 is kicked out of memory (but put on disk) while list4 is added val updatedBlocks4 = - store.put("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks4.size === 2) updatedBlocks4.foreach { case (id, status) => id match { @@ -916,26 +889,37 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter case _ => fail("Updated block is neither list2 nor list4") } } - assert(store.get("list4").isDefined, "list4 was not in store") + assert(store.diskStore.contains("list2"), "list2 was not in disk store") + assert(store.memoryStore.contains("list4"), "list4 was not in memory store") - // No updated blocks - nothing is kicked out of memory because list5 is too big to be added + // No updated blocks - list5 is too big to fit in store and nothing is kicked out val updatedBlocks5 = - store.put("list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks5.size === 0) - assert(store.get("list2").isDefined, "list2 was not in store") - assert(store.get("list4").isDefined, "list4 was not in store") - assert(!store.get("list5").isDefined, "list5 was in store") + + // memory store contains only list3 and list4 + assert(!store.memoryStore.contains("list1"), "list1 was in memory store") + assert(!store.memoryStore.contains("list2"), "list2 was in memory store") + assert(store.memoryStore.contains("list3"), "list3 was not in memory store") + assert(store.memoryStore.contains("list4"), "list4 was not in memory store") + assert(!store.memoryStore.contains("list5"), "list5 was in memory store") + + // disk store contains only list2 + assert(!store.diskStore.contains("list1"), "list1 was in disk store") + assert(store.diskStore.contains("list2"), "list2 was not in disk store") + assert(!store.diskStore.contains("list3"), "list3 was in disk store") + assert(!store.diskStore.contains("list4"), "list4 was in disk store") + assert(!store.diskStore.contains("list5"), "list5 was in disk store") } test("query block statuses") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list = List.fill(2)(new Array[Byte](200)) + store = makeBlockManager(12000) + val list = List.fill(2)(new Array[Byte](2000)) // Tell master. By LRU, only list2 and list3 remains. - store.put("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.put("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) // getLocations and getBlockStatus should yield the same locations assert(store.master.getLocations("list1").size === 0) @@ -949,9 +933,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.master.getBlockStatus("list3", askSlaves = true).size === 1) // This time don't tell master and see what happens. By LRU, only list5 and list6 remains. - store.put("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) - store.put("list5", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - store.put("list6", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + store.putIterator("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + store.putIterator("list5", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIterator("list6", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) // getLocations should return nothing because the master is not informed // getBlockStatus without asking slaves should have the same result @@ -968,23 +952,22 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("get matching blocks") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list = List.fill(2)(new Array[Byte](10)) + store = makeBlockManager(12000) + val list = List.fill(2)(new Array[Byte](100)) // insert some blocks - store.put("list1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.put("list3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) // getLocations and getBlockStatus should yield the same locations assert(store.master.getMatchingBlockIds(_.toString.contains("list"), askSlaves = false).size === 3) assert(store.master.getMatchingBlockIds(_.toString.contains("list1"), askSlaves = false).size === 1) // insert some more blocks - store.put("newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.put("newlist2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - store.put("newlist3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIterator("newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("newlist2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIterator("newlist3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) // getLocations and getBlockStatus should yield the same locations assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = false).size === 1) @@ -992,7 +975,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter val blockIds = Seq(RDDBlockId(1, 0), RDDBlockId(1, 1), RDDBlockId(2, 0)) blockIds.foreach { blockId => - store.put(blockId, list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator(blockId, list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } val matchedBlockIds = store.master.getMatchingBlockIds(_ match { case RDDBlockId(1, _) => true @@ -1002,17 +985,240 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - store.putSingle(rdd(0, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(1, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store = makeBlockManager(12000) + store.putSingle(rdd(0, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(1, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // Access rdd_1_0 to ensure it's not least recently used. assert(store.getSingle(rdd(1, 0)).isDefined, "rdd_1_0 was not in store") // According to the same-RDD rule, rdd_1_0 should be replaced here. - store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(0, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // rdd_1_0 should have been replaced, even it's not least recently used. assert(store.memoryStore.contains(rdd(0, 0)), "rdd_0_0 was not in store") assert(store.memoryStore.contains(rdd(0, 1)), "rdd_0_1 was not in store") assert(!store.memoryStore.contains(rdd(1, 0)), "rdd_1_0 was in store") } + + test("reserve/release unroll memory") { + store = makeBlockManager(12000) + val memoryStore = store.memoryStore + assert(memoryStore.currentUnrollMemory === 0) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Reserve + memoryStore.reserveUnrollMemoryForThisThread(100) + assert(memoryStore.currentUnrollMemoryForThisThread === 100) + memoryStore.reserveUnrollMemoryForThisThread(200) + assert(memoryStore.currentUnrollMemoryForThisThread === 300) + memoryStore.reserveUnrollMemoryForThisThread(500) + assert(memoryStore.currentUnrollMemoryForThisThread === 800) + memoryStore.reserveUnrollMemoryForThisThread(1000000) + assert(memoryStore.currentUnrollMemoryForThisThread === 800) // not granted + // Release + memoryStore.releaseUnrollMemoryForThisThread(100) + assert(memoryStore.currentUnrollMemoryForThisThread === 700) + memoryStore.releaseUnrollMemoryForThisThread(100) + assert(memoryStore.currentUnrollMemoryForThisThread === 600) + // Reserve again + memoryStore.reserveUnrollMemoryForThisThread(4400) + assert(memoryStore.currentUnrollMemoryForThisThread === 5000) + memoryStore.reserveUnrollMemoryForThisThread(20000) + assert(memoryStore.currentUnrollMemoryForThisThread === 5000) // not granted + // Release again + memoryStore.releaseUnrollMemoryForThisThread(1000) + assert(memoryStore.currentUnrollMemoryForThisThread === 4000) + memoryStore.releaseUnrollMemoryForThisThread() // release all + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + } + + /** + * Verify the result of MemoryStore#unrollSafely is as expected. + */ + private def verifyUnroll( + expected: Iterator[Any], + result: Either[Array[Any], Iterator[Any]], + shouldBeArray: Boolean): Unit = { + val actual: Iterator[Any] = result match { + case Left(arr: Array[Any]) => + assert(shouldBeArray, "expected iterator from unroll!") + arr.iterator + case Right(it: Iterator[Any]) => + assert(!shouldBeArray, "expected array from unroll!") + it + case _ => + fail("unroll returned neither an iterator nor an array...") + } + expected.zip(actual).foreach { case (e, a) => + assert(e === a, "unroll did not return original values!") + } + } + + test("safely unroll blocks") { + store = makeBlockManager(12000) + val smallList = List.fill(40)(new Array[Byte](100)) + val bigList = List.fill(40)(new Array[Byte](1000)) + val memoryStore = store.memoryStore + val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Unroll with all the space in the world. This should succeed and return an array. + var unrollResult = memoryStore.unrollSafely("unroll", smallList.iterator, droppedBlocks) + verifyUnroll(smallList.iterator, unrollResult, shouldBeArray = true) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Unroll with not enough space. This should succeed after kicking out someBlock1. + store.putIterator("someBlock1", smallList.iterator, StorageLevel.MEMORY_ONLY) + store.putIterator("someBlock2", smallList.iterator, StorageLevel.MEMORY_ONLY) + unrollResult = memoryStore.unrollSafely("unroll", smallList.iterator, droppedBlocks) + verifyUnroll(smallList.iterator, unrollResult, shouldBeArray = true) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + assert(droppedBlocks.size === 1) + assert(droppedBlocks.head._1 === TestBlockId("someBlock1")) + droppedBlocks.clear() + + // Unroll huge block with not enough space. Even after ensuring free space of 12000 * 0.4 = + // 4800 bytes, there is still not enough room to unroll this block. This returns an iterator. + // In the mean time, however, we kicked out someBlock2 before giving up. + store.putIterator("someBlock3", smallList.iterator, StorageLevel.MEMORY_ONLY) + unrollResult = memoryStore.unrollSafely("unroll", bigList.iterator, droppedBlocks) + verifyUnroll(bigList.iterator, unrollResult, shouldBeArray = false) + assert(memoryStore.currentUnrollMemoryForThisThread > 0) // we returned an iterator + assert(droppedBlocks.size === 1) + assert(droppedBlocks.head._1 === TestBlockId("someBlock2")) + droppedBlocks.clear() + } + + test("safely unroll blocks through putIterator") { + store = makeBlockManager(12000) + val memOnly = StorageLevel.MEMORY_ONLY + val memoryStore = store.memoryStore + val smallList = List.fill(40)(new Array[Byte](100)) + val bigList = List.fill(40)(new Array[Byte](1000)) + def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] + def bigIterator = bigList.iterator.asInstanceOf[Iterator[Any]] + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Unroll with plenty of space. This should succeed and cache both blocks. + val result1 = memoryStore.putIterator("b1", smallIterator, memOnly, returnValues = true) + val result2 = memoryStore.putIterator("b2", smallIterator, memOnly, returnValues = true) + assert(memoryStore.contains("b1")) + assert(memoryStore.contains("b2")) + assert(result1.size > 0) // unroll was successful + assert(result2.size > 0) + assert(result1.data.isLeft) // unroll did not drop this block to disk + assert(result2.data.isLeft) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Re-put these two blocks so block manager knows about them too. Otherwise, block manager + // would not know how to drop them from memory later. + memoryStore.remove("b1") + memoryStore.remove("b2") + store.putIterator("b1", smallIterator, memOnly) + store.putIterator("b2", smallIterator, memOnly) + + // Unroll with not enough space. This should succeed but kick out b1 in the process. + val result3 = memoryStore.putIterator("b3", smallIterator, memOnly, returnValues = true) + assert(result3.size > 0) + assert(result3.data.isLeft) + assert(!memoryStore.contains("b1")) + assert(memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + memoryStore.remove("b3") + store.putIterator("b3", smallIterator, memOnly) + + // Unroll huge block with not enough space. This should fail and kick out b2 in the process. + val result4 = memoryStore.putIterator("b4", bigIterator, memOnly, returnValues = true) + assert(result4.size === 0) // unroll was unsuccessful + assert(result4.data.isLeft) + assert(!memoryStore.contains("b1")) + assert(!memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(!memoryStore.contains("b4")) + assert(memoryStore.currentUnrollMemoryForThisThread > 0) // we returned an iterator + } + + /** + * This test is essentially identical to the preceding one, except that it uses MEMORY_AND_DISK. + */ + test("safely unroll blocks through putIterator (disk)") { + store = makeBlockManager(12000) + val memAndDisk = StorageLevel.MEMORY_AND_DISK + val memoryStore = store.memoryStore + val diskStore = store.diskStore + val smallList = List.fill(40)(new Array[Byte](100)) + val bigList = List.fill(40)(new Array[Byte](1000)) + def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] + def bigIterator = bigList.iterator.asInstanceOf[Iterator[Any]] + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + store.putIterator("b1", smallIterator, memAndDisk) + store.putIterator("b2", smallIterator, memAndDisk) + + // Unroll with not enough space. This should succeed but kick out b1 in the process. + // Memory store should contain b2 and b3, while disk store should contain only b1 + val result3 = memoryStore.putIterator("b3", smallIterator, memAndDisk, returnValues = true) + assert(result3.size > 0) + assert(!memoryStore.contains("b1")) + assert(memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(diskStore.contains("b1")) + assert(!diskStore.contains("b2")) + assert(!diskStore.contains("b3")) + memoryStore.remove("b3") + store.putIterator("b3", smallIterator, StorageLevel.MEMORY_ONLY) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Unroll huge block with not enough space. This should fail and drop the new block to disk + // directly in addition to kicking out b2 in the process. Memory store should contain only + // b3, while disk store should contain b1, b2 and b4. + val result4 = memoryStore.putIterator("b4", bigIterator, memAndDisk, returnValues = true) + assert(result4.size > 0) + assert(result4.data.isRight) // unroll returned bytes from disk + assert(!memoryStore.contains("b1")) + assert(!memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(!memoryStore.contains("b4")) + assert(diskStore.contains("b1")) + assert(diskStore.contains("b2")) + assert(!diskStore.contains("b3")) + assert(diskStore.contains("b4")) + assert(memoryStore.currentUnrollMemoryForThisThread > 0) // we returned an iterator + } + + test("multiple unrolls by the same thread") { + store = makeBlockManager(12000) + val memOnly = StorageLevel.MEMORY_ONLY + val memoryStore = store.memoryStore + val smallList = List.fill(40)(new Array[Byte](100)) + def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // All unroll memory used is released because unrollSafely returned an array + memoryStore.putIterator("b1", smallIterator, memOnly, returnValues = true) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + memoryStore.putIterator("b2", smallIterator, memOnly, returnValues = true) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Unroll memory is not released because unrollSafely returned an iterator + // that still depends on the underlying vector used in the process + memoryStore.putIterator("b3", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB3 = memoryStore.currentUnrollMemoryForThisThread + assert(unrollMemoryAfterB3 > 0) + + // The unroll memory owned by this thread builds on top of its value after the previous unrolls + memoryStore.putIterator("b4", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB4 = memoryStore.currentUnrollMemoryForThisThread + assert(unrollMemoryAfterB4 > unrollMemoryAfterB3) + + // ... but only to a certain extent (until we run out of free space to grant new unroll memory) + memoryStore.putIterator("b5", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB5 = memoryStore.currentUnrollMemoryForThisThread + memoryStore.putIterator("b6", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB6 = memoryStore.currentUnrollMemoryForThisThread + memoryStore.putIterator("b7", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB7 = memoryStore.currentUnrollMemoryForThisThread + assert(unrollMemoryAfterB5 === unrollMemoryAfterB4) + assert(unrollMemoryAfterB6 === unrollMemoryAfterB4) + assert(unrollMemoryAfterB7 === unrollMemoryAfterB4) + } } diff --git a/core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala deleted file mode 100644 index 93f0c6a8e6408..0000000000000 --- a/core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala +++ /dev/null @@ -1,120 +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.util - -import scala.util.Random - -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import org.apache.spark.util.SizeTrackingAppendOnlyMapSuite.LargeDummyClass -import org.apache.spark.util.collection.{AppendOnlyMap, SizeTrackingAppendOnlyMap} - -class SizeTrackingAppendOnlyMapSuite extends FunSuite with BeforeAndAfterAll { - val NORMAL_ERROR = 0.20 - val HIGH_ERROR = 0.30 - - test("fixed size insertions") { - testWith[Int, Long](10000, i => (i, i.toLong)) - testWith[Int, (Long, Long)](10000, i => (i, (i.toLong, i.toLong))) - testWith[Int, LargeDummyClass](10000, i => (i, new LargeDummyClass())) - } - - test("variable size insertions") { - val rand = new Random(123456789) - def randString(minLen: Int, maxLen: Int): String = { - "a" * (rand.nextInt(maxLen - minLen) + minLen) - } - testWith[Int, String](10000, i => (i, randString(0, 10))) - testWith[Int, String](10000, i => (i, randString(0, 100))) - testWith[Int, String](10000, i => (i, randString(90, 100))) - } - - test("updates") { - val rand = new Random(123456789) - def randString(minLen: Int, maxLen: Int): String = { - "a" * (rand.nextInt(maxLen - minLen) + minLen) - } - testWith[String, Int](10000, i => (randString(0, 10000), i)) - } - - def testWith[K, V](numElements: Int, makeElement: (Int) => (K, V)) { - val map = new SizeTrackingAppendOnlyMap[K, V]() - for (i <- 0 until numElements) { - val (k, v) = makeElement(i) - map(k) = v - expectWithinError(map, map.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) - } - } - - def expectWithinError(obj: AnyRef, estimatedSize: Long, error: Double) { - val betterEstimatedSize = SizeEstimator.estimate(obj) - assert(betterEstimatedSize * (1 - error) < estimatedSize, - s"Estimated size $estimatedSize was less than expected size $betterEstimatedSize") - assert(betterEstimatedSize * (1 + 2 * error) > estimatedSize, - s"Estimated size $estimatedSize was greater than expected size $betterEstimatedSize") - } -} - -object SizeTrackingAppendOnlyMapSuite { - // Speed test, for reproducibility of results. - // These could be highly non-deterministic in general, however. - // Results: - // AppendOnlyMap: 31 ms - // SizeTracker: 54 ms - // SizeEstimator: 1500 ms - def main(args: Array[String]) { - val numElements = 100000 - - val baseTimes = for (i <- 0 until 10) yield time { - val map = new AppendOnlyMap[Int, LargeDummyClass]() - for (i <- 0 until numElements) { - map(i) = new LargeDummyClass() - } - } - - val sampledTimes = for (i <- 0 until 10) yield time { - val map = new SizeTrackingAppendOnlyMap[Int, LargeDummyClass]() - for (i <- 0 until numElements) { - map(i) = new LargeDummyClass() - map.estimateSize() - } - } - - val unsampledTimes = for (i <- 0 until 3) yield time { - val map = new AppendOnlyMap[Int, LargeDummyClass]() - for (i <- 0 until numElements) { - map(i) = new LargeDummyClass() - SizeEstimator.estimate(map) - } - } - - println("Base: " + baseTimes) - println("SizeTracker (sampled): " + sampledTimes) - println("SizeEstimator (unsampled): " + unsampledTimes) - } - - def time(f: => Unit): Long = { - val start = System.currentTimeMillis() - f - System.currentTimeMillis() - start - } - - private class LargeDummyClass { - val arr = new Array[Int](100) - } -} diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 428822949c085..0b7ad184a46d2 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -63,12 +63,13 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, mergeValue, mergeCombiners) - map.insert(1, 10) - map.insert(2, 20) - map.insert(3, 30) - map.insert(1, 100) - map.insert(2, 200) - map.insert(1, 1000) + map.insertAll(Seq( + (1, 10), + (2, 20), + (3, 30), + (1, 100), + (2, 200), + (1, 1000))) val it = map.iterator assert(it.hasNext) val result = it.toSet[(Int, ArrayBuffer[Int])].map(kv => (kv._1, kv._2.toSet)) @@ -282,7 +283,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { assert(w1.hashCode === w2.hashCode) } - (1 to 100000).map(_.toString).foreach { i => map.insert(i, i) } + map.insertAll((1 to 100000).iterator.map(_.toString).map(i => (i, i))) collisionPairs.foreach { case (w1, w2) => map.insert(w1, w2) map.insert(w2, w1) @@ -355,7 +356,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]]( createCombiner, mergeValue, mergeCombiners) - (1 to 100000).foreach { i => map.insert(i, i) } + map.insertAll((1 to 100000).iterator.map(i => (i, i))) map.insert(null.asInstanceOf[Int], 1) map.insert(1, null.asInstanceOf[Int]) map.insert(null.asInstanceOf[Int], null.asInstanceOf[Int]) diff --git a/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala new file mode 100644 index 0000000000000..1f33967249654 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala @@ -0,0 +1,204 @@ +/* + * 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.util.collection + +import scala.reflect.ClassTag +import scala.util.Random + +import org.scalatest.FunSuite + +import org.apache.spark.util.SizeEstimator + +class SizeTrackerSuite extends FunSuite { + val NORMAL_ERROR = 0.20 + val HIGH_ERROR = 0.30 + + import SizeTrackerSuite._ + + test("vector fixed size insertions") { + testVector[Long](10000, i => i.toLong) + testVector[(Long, Long)](10000, i => (i.toLong, i.toLong)) + testVector[LargeDummyClass](10000, i => new LargeDummyClass) + } + + test("vector variable size insertions") { + val rand = new Random(123456789) + def randString(minLen: Int, maxLen: Int): String = { + "a" * (rand.nextInt(maxLen - minLen) + minLen) + } + testVector[String](10000, i => randString(0, 10)) + testVector[String](10000, i => randString(0, 100)) + testVector[String](10000, i => randString(90, 100)) + } + + test("map fixed size insertions") { + testMap[Int, Long](10000, i => (i, i.toLong)) + testMap[Int, (Long, Long)](10000, i => (i, (i.toLong, i.toLong))) + testMap[Int, LargeDummyClass](10000, i => (i, new LargeDummyClass)) + } + + test("map variable size insertions") { + val rand = new Random(123456789) + def randString(minLen: Int, maxLen: Int): String = { + "a" * (rand.nextInt(maxLen - minLen) + minLen) + } + testMap[Int, String](10000, i => (i, randString(0, 10))) + testMap[Int, String](10000, i => (i, randString(0, 100))) + testMap[Int, String](10000, i => (i, randString(90, 100))) + } + + test("map updates") { + val rand = new Random(123456789) + def randString(minLen: Int, maxLen: Int): String = { + "a" * (rand.nextInt(maxLen - minLen) + minLen) + } + testMap[String, Int](10000, i => (randString(0, 10000), i)) + } + + def testVector[T: ClassTag](numElements: Int, makeElement: Int => T) { + val vector = new SizeTrackingVector[T] + for (i <- 0 until numElements) { + val item = makeElement(i) + vector += item + expectWithinError(vector, vector.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) + } + } + + def testMap[K, V](numElements: Int, makeElement: (Int) => (K, V)) { + val map = new SizeTrackingAppendOnlyMap[K, V] + for (i <- 0 until numElements) { + val (k, v) = makeElement(i) + map(k) = v + expectWithinError(map, map.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) + } + } + + def expectWithinError(obj: AnyRef, estimatedSize: Long, error: Double) { + val betterEstimatedSize = SizeEstimator.estimate(obj) + assert(betterEstimatedSize * (1 - error) < estimatedSize, + s"Estimated size $estimatedSize was less than expected size $betterEstimatedSize") + assert(betterEstimatedSize * (1 + 2 * error) > estimatedSize, + s"Estimated size $estimatedSize was greater than expected size $betterEstimatedSize") + } +} + +private object SizeTrackerSuite { + + /** + * Run speed tests for size tracking collections. + */ + def main(args: Array[String]): Unit = { + if (args.size < 1) { + println("Usage: SizeTrackerSuite [num elements]") + System.exit(1) + } + val numElements = args(0).toInt + vectorSpeedTest(numElements) + mapSpeedTest(numElements) + } + + /** + * Speed test for SizeTrackingVector. + * + * Results for 100000 elements (possibly non-deterministic): + * PrimitiveVector 15 ms + * SizeTracker 51 ms + * SizeEstimator 2000 ms + */ + def vectorSpeedTest(numElements: Int): Unit = { + val baseTimes = for (i <- 0 until 10) yield time { + val vector = new PrimitiveVector[LargeDummyClass] + for (i <- 0 until numElements) { + vector += new LargeDummyClass + } + } + val sampledTimes = for (i <- 0 until 10) yield time { + val vector = new SizeTrackingVector[LargeDummyClass] + for (i <- 0 until numElements) { + vector += new LargeDummyClass + vector.estimateSize() + } + } + val unsampledTimes = for (i <- 0 until 3) yield time { + val vector = new PrimitiveVector[LargeDummyClass] + for (i <- 0 until numElements) { + vector += new LargeDummyClass + SizeEstimator.estimate(vector) + } + } + printSpeedTestResult("SizeTrackingVector", baseTimes, sampledTimes, unsampledTimes) + } + + /** + * Speed test for SizeTrackingAppendOnlyMap. + * + * Results for 100000 elements (possibly non-deterministic): + * AppendOnlyMap 30 ms + * SizeTracker 41 ms + * SizeEstimator 1666 ms + */ + def mapSpeedTest(numElements: Int): Unit = { + val baseTimes = for (i <- 0 until 10) yield time { + val map = new AppendOnlyMap[Int, LargeDummyClass] + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass + } + } + val sampledTimes = for (i <- 0 until 10) yield time { + val map = new SizeTrackingAppendOnlyMap[Int, LargeDummyClass] + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass + map.estimateSize() + } + } + val unsampledTimes = for (i <- 0 until 3) yield time { + val map = new AppendOnlyMap[Int, LargeDummyClass] + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass + SizeEstimator.estimate(map) + } + } + printSpeedTestResult("SizeTrackingAppendOnlyMap", baseTimes, sampledTimes, unsampledTimes) + } + + def printSpeedTestResult( + testName: String, + baseTimes: Seq[Long], + sampledTimes: Seq[Long], + unsampledTimes: Seq[Long]): Unit = { + println(s"Average times for $testName (ms):") + println(" Base - " + averageTime(baseTimes)) + println(" SizeTracker (sampled) - " + averageTime(sampledTimes)) + println(" SizeEstimator (unsampled) - " + averageTime(unsampledTimes)) + println() + } + + def time(f: => Unit): Long = { + val start = System.currentTimeMillis() + f + System.currentTimeMillis() - start + } + + def averageTime(v: Seq[Long]): Long = { + v.sum / v.size + } + + private class LargeDummyClass { + val arr = new Array[Int](100) + } +} diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 33de24d1ae6d7..38830103d1e8d 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -53,7 +53,7 @@ if [[ ! "$@" =~ --package-only ]]; then -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ -Dmaven.javadoc.skip=true \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ --batch-mode release:prepare @@ -61,7 +61,7 @@ if [[ ! "$@" =~ --package-only ]]; then -Darguments="-DskipTests=true -Dmaven.javadoc.skip=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 \ -Dmaven.javadoc.skip=true \ - -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ release:perform cd .. @@ -111,10 +111,10 @@ make_binary_release() { spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } -make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" -make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" +make_binary_release "hadoop1" "-Phive -Dhadoop.version=1.0.4" +make_binary_release "cdh4" "-Phive -Dhadoop.version=2.0.0-mr1-cdh4.2.0" make_binary_release "hadoop2" \ - "-Phive -Phive-thriftserver -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" + "-Phive -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" # Copy data echo "Copying release tarballs" diff --git a/dev/run-tests b/dev/run-tests index 98ec969dc1b37..51e4def0f835a 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -65,7 +65,7 @@ echo "=========================================================================" # (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. if [ -n "$_RUN_SQL_TESTS" ]; then - echo -e "q\n" | SBT_MAVEN_PROFILES="$SBT_MAVEN_PROFILES -Phive -Phive-thriftserver" sbt/sbt clean package \ + echo -e "q\n" | SBT_MAVEN_PROFILES="$SBT_MAVEN_PROFILES -Phive" sbt/sbt clean package \ assembly/assembly test | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" else echo -e "q\n" | sbt/sbt clean package assembly/assembly test | \ diff --git a/dev/scalastyle b/dev/scalastyle index d9f2b91a3a091..a02d06912f238 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,7 +17,7 @@ # limitations under the License. # -echo -e "q\n" | sbt/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt +echo -e "q\n" | sbt/sbt -Phive scalastyle > scalastyle.txt # Check style with YARN alpha built too echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ >> scalastyle.txt diff --git a/docs/configuration.md b/docs/configuration.md index dac8bb1d52468..2e6c85cc2bcca 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -239,7 +239,7 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.memoryFraction - 0.3 + 0.2 Fraction of Java heap to use for aggregation and cogroups during shuffles, if spark.shuffle.spill is true. At any given time, the collective size of @@ -380,13 +380,13 @@ Apart from these, the following properties are also available, and may be useful spark.serializer.objectStreamReset - 10000 + 100 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 <= 0. - By default it will reset the serializer every 10,000 objects. + By default it will reset the serializer every 100 objects. @@ -480,6 +480,15 @@ Apart from these, the following properties are also available, and may be useful increase it if you configure your own old generation size. + + spark.storage.unrollFraction + 0.2 + + Fraction of spark.storage.memoryFraction to use for unrolling blocks in memory. + This is dynamically allocated by dropping existing blocks when there is not enough free + storage space to unroll the new block in its entirety. + + spark.tachyonStore.baseDir System.getProperty("java.io.tmpdir") diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 36d642f2923b2..38728534a46e0 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -136,7 +136,7 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext.createSchemaRDD // Define the schema using a case class. -// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, +// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, // you can use custom classes that implement the Product interface. case class Person(name: String, age: Int) @@ -548,6 +548,7 @@ results = hiveContext.hql("FROM src SELECT key, value").collect() + # Writing Language-Integrated Relational Queries **Language-Integrated queries are currently only supported in Scala.** @@ -572,199 +573,4 @@ prefixed with a tick (`'`). Implicit conversions turn these symbols into expres evaluated by the SQL execution engine. A full list of the functions supported can be found in the [ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). - - -## Running the Thrift JDBC server - -The Thrift JDBC server implemented here corresponds to the [`HiveServer2`] -(https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) in Hive 0.12. You can test -the JDBC server with the beeline script comes with either Spark or Hive 0.12. In order to use Hive -you must first run '`sbt/sbt -Phive-thriftserver assembly/assembly`' (or use `-Phive-thriftserver` -for maven). - -To start the JDBC server, run the following in the Spark directory: - - ./sbin/start-thriftserver.sh - -The default port the server listens on is 10000. You may run -`./sbin/start-thriftserver.sh --help` for a complete list of all available -options. Now you can use beeline to test the Thrift JDBC server: - - ./bin/beeline - -Connect to the JDBC server in beeline with: - - beeline> !connect jdbc:hive2://localhost:10000 - -Beeline will ask you for a username and password. In non-secure mode, simply enter the username on -your machine and a blank password. For secure mode, please follow the instructions given in the -[beeline documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients) - -Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. - -You may also use the beeline script comes with Hive. - -### Migration Guide for Shark Users - -#### Reducer number - -In Shark, default reducer number is 1 and is controlled by the property `mapred.reduce.tasks`. Spark -SQL deprecates this property by a new property `spark.sql.shuffle.partitions`, whose default value -is 200. Users may customize this property via `SET`: - -``` -SET spark.sql.shuffle.partitions=10; -SELECT page, count(*) c FROM logs_last_month_cached -GROUP BY page ORDER BY c DESC LIMIT 10; -``` - -You may also put this property in `hive-site.xml` to override the default value. - -For now, the `mapred.reduce.tasks` property is still recognized, and is converted to -`spark.sql.shuffle.partitions` automatically. - -#### Caching - -The `shark.cache` table property no longer exists, and tables whose name end with `_cached` are no -longer automcatically cached. Instead, we provide `CACHE TABLE` and `UNCACHE TABLE` statements to -let user control table caching explicitly: - -``` -CACHE TABLE logs_last_month; -UNCACHE TABLE logs_last_month; -``` - -**NOTE** `CACHE TABLE tbl` is lazy, it only marks table `tbl` as "need to by cached if necessary", -but doesn't actually cache it until a query that touches `tbl` is executed. To force the table to be -cached, you may simply count the table immediately after executing `CACHE TABLE`: - -``` -CACHE TABLE logs_last_month; -SELECT COUNT(1) FROM logs_last_month; -``` - -Several caching related features are not supported yet: - -* User defined partition level cache eviction policy -* RDD reloading -* In-memory cache write through policy - -### Compatibility with Apache Hive - -#### Deploying in Exising Hive Warehouses - -Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive -installations. You do not need to modify your existing Hive Metastore or change the data placement -or partitioning of your tables. - -#### Supported Hive Features - -Spark SQL supports the vast majority of Hive features, such as: - -* Hive query statements, including: - * `SELECT` - * `GROUP BY - * `ORDER BY` - * `CLUSTER BY` - * `SORT BY` -* All Hive operators, including: - * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) - * Arthimatic operators (`+`, `-`, `*`, `/`, `%`, etc) - * Logical operators (`AND`, `&&`, `OR`, `||`, etc) - * Complex type constructors - * Mathemtatical functions (`sign`, `ln`, `cos`, etc) - * String functions (`instr`, `length`, `printf`, etc) -* User defined functions (UDF) -* User defined aggregation functions (UDAF) -* User defined serialization formats (SerDe's) -* Joins - * `JOIN` - * `{LEFT|RIGHT|FULL} OUTER JOIN` - * `LEFT SEMI JOIN` - * `CROSS JOIN` -* Unions -* Sub queries - * `SELECT col FROM ( SELECT a + b AS col from t1) t2` -* Sampling -* Explain -* Partitioned tables -* All Hive DDL Functions, including: - * `CREATE TABLE` - * `CREATE TABLE AS SELECT` - * `ALTER TABLE` -* Most Hive Data types, including: - * `TINYINT` - * `SMALLINT` - * `INT` - * `BIGINT` - * `BOOLEAN` - * `FLOAT` - * `DOUBLE` - * `STRING` - * `BINARY` - * `TIMESTAMP` - * `ARRAY<>` - * `MAP<>` - * `STRUCT<>` - -#### Unsupported Hive Functionality - -Below is a list of Hive features that we don't support yet. Most of these features are rarely used -in Hive deployments. - -**Major Hive Features** - -* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL - doesn't support buckets yet. - -**Esoteric Hive Features** - -* Tables with partitions using different input formats: In Spark SQL, all table partitions need to - have the same input format. -* Non-equi outer join: For the uncommon use case of using outer joins with non-equi join conditions - (e.g. condition "`key < 10`"), Spark SQL will output wrong result for the `NULL` tuple. -* `UNIONTYPE` -* Unique join -* Single query multi insert -* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at - the moment. - -**Hive Input/Output Formats** - -* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. -* Hadoop archive - -**Hive Optimizations** - -A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are -not necessary due to Spark SQL's in-memory computational model. Others are slotted for future -releases of Spark SQL. - -* Block level bitmap indexes and virtual columns (used to build indexes) -* Automatically convert a join to map join: For joining a large table with multiple small tables, - Hive automatically converts the join into a map join. We are adding this auto conversion in the - next release. -* Automatically determine the number of reducers for joins and groupbys: Currently in Spark SQL, you - need to control the degree of parallelism post-shuffle using "SET - spark.sql.shuffle.partitions=[num_tasks];". We are going to add auto-setting of parallelism in the - next release. -* Meta-data only query: For queries that can be answered by using only meta data, Spark SQL still - launches tasks to compute the result. -* Skew data flag: Spark SQL does not follow the skew data flags in Hive. -* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. -* Merge multiple small files for query results: if the result output contains multiple small files, - Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS - metadata. Spark SQL does not support that. - -## Running the Spark SQL CLI - -The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute -queries input from command line. Note: the Spark SQL CLI cannot talk to the Thrift JDBC server. - -To start the Spark SQL CLI, run the following in the Spark directory: - - ./bin/spark-sql - -Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. -You may run `./bin/spark-sql --help` for a complete list of all available -options. + \ No newline at end of file diff --git a/examples/pom.xml b/examples/pom.xml index c4ed0f5a6a02b..bd1c387c2eb91 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-examples_2.10 - examples + examples jar Spark Project Examples diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 9f680b27c3308..e6b3cc36702c8 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-flume_2.10 - streaming-flume + streaming-flume jar Spark Project External Flume diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 25a5c0a4d7d77..4762c50685a93 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-kafka_2.10 - streaming-kafka + streaming-kafka jar Spark Project External Kafka diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index f31ed655f6779..32c530e600ce0 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-mqtt_2.10 - streaming-mqtt + streaming-mqtt jar Spark Project External MQTT diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 56bb24c2a072e..637adb0f00da0 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-twitter_2.10 - streaming-twitter + streaming-twitter jar Spark Project External Twitter diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 54b0242c54e78..e4d758a04a4cd 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-zeromq_2.10 - streaming-zeromq + streaming-zeromq jar Spark Project External ZeroMQ diff --git a/graphx/pom.xml b/graphx/pom.xml index 6dd52fc618b1e..7e3bcf29dcfbc 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-graphx_2.10 - graphx + graphx jar Spark Project GraphX diff --git a/mllib/pom.xml b/mllib/pom.xml index f27cf520dc9fa..92b07e2357db1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-mllib_2.10 - mllib + mllib jar Spark Project ML Library diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index c44173793b39a..954621ee8b933 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -54,6 +54,13 @@ class PythonMLLibAPI extends Serializable { } } + private[python] def deserializeDouble(bytes: Array[Byte], offset: Int = 0): Double = { + require(bytes.length - offset == 8, "Wrong size byte array for Double") + val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) + bb.order(ByteOrder.nativeOrder()) + bb.getDouble + } + private def deserializeDenseVector(bytes: Array[Byte], offset: Int = 0): Vector = { val packetLength = bytes.length - offset require(packetLength >= 5, "Byte array too short") @@ -89,6 +96,22 @@ class PythonMLLibAPI extends Serializable { Vectors.sparse(size, indices, values) } + /** + * Returns an 8-byte array for the input Double. + * + * Note: we currently do not use a magic byte for double for storage efficiency. + * This should be reconsidered when we add Ser/De for other 8-byte types (e.g. Long), for safety. + * The corresponding deserializer, deserializeDouble, needs to be modified as well if the + * serialization scheme changes. + */ + private[python] def serializeDouble(double: Double): Array[Byte] = { + val bytes = new Array[Byte](8) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + bb.putDouble(double) + bytes + } + private def serializeDenseVector(doubles: Array[Double]): Array[Byte] = { val len = doubles.length val bytes = new Array[Byte](5 + 8 * len) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index b6e0c4a80e27b..6c7be0a4f1dcb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -54,7 +54,13 @@ class NaiveBayesModel private[mllib] ( } } - override def predict(testData: RDD[Vector]): RDD[Double] = testData.map(predict) + override def predict(testData: RDD[Vector]): RDD[Double] = { + val bcModel = testData.context.broadcast(this) + testData.mapPartitions { iter => + val model = bcModel.value + iter.map(model.predict) + } + } override def predict(testData: Vector): Double = { labels(brzArgmax(brzPi + brzTheta * testData.toBreeze)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index de22fbb6ffc10..db425d866bbad 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -165,18 +165,21 @@ class KMeans private ( val activeCenters = activeRuns.map(r => centers(r)).toArray val costAccums = activeRuns.map(_ => sc.accumulator(0.0)) + val bcActiveCenters = sc.broadcast(activeCenters) + // Find the sum and count of points mapping to each center val totalContribs = data.mapPartitions { points => - val runs = activeCenters.length - val k = activeCenters(0).length - val dims = activeCenters(0)(0).vector.length + val thisActiveCenters = bcActiveCenters.value + val runs = thisActiveCenters.length + val k = thisActiveCenters(0).length + val dims = thisActiveCenters(0)(0).vector.length val sums = Array.fill(runs, k)(BDV.zeros[Double](dims).asInstanceOf[BV[Double]]) val counts = Array.fill(runs, k)(0L) points.foreach { point => (0 until runs).foreach { i => - val (bestCenter, cost) = KMeans.findClosest(activeCenters(i), point) + val (bestCenter, cost) = KMeans.findClosest(thisActiveCenters(i), point) costAccums(i) += cost sums(i)(bestCenter) += point.vector counts(i)(bestCenter) += 1 @@ -264,16 +267,17 @@ class KMeans private ( // to their squared distance from that run's current centers var step = 0 while (step < initializationSteps) { + val bcCenters = data.context.broadcast(centers) val sumCosts = data.flatMap { point => (0 until runs).map { r => - (r, KMeans.pointCost(centers(r), point)) + (r, KMeans.pointCost(bcCenters.value(r), point)) } }.reduceByKey(_ + _).collectAsMap() val chosen = data.mapPartitionsWithIndex { (index, points) => val rand = new XORShiftRandom(seed ^ (step << 16) ^ index) points.flatMap { p => (0 until runs).filter { r => - rand.nextDouble() < 2.0 * KMeans.pointCost(centers(r), p) * k / sumCosts(r) + rand.nextDouble() < 2.0 * KMeans.pointCost(bcCenters.value(r), p) * k / sumCosts(r) }.map((_, p)) } }.collect() @@ -286,9 +290,10 @@ class KMeans private ( // Finally, we might have a set of more than k candidate centers for each run; weigh each // candidate by the number of points in the dataset mapping to it and run a local k-means++ // on the weighted centers to pick just k of them + val bcCenters = data.context.broadcast(centers) val weightMap = data.flatMap { p => (0 until runs).map { r => - ((r, KMeans.findClosest(centers(r), p)._1), 1.0) + ((r, KMeans.findClosest(bcCenters.value(r), p)._1), 1.0) } }.reduceByKey(_ + _).collectAsMap() val finalCenters = (0 until runs).map { r => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index fba21aefaaacd..5823cb6e52e7f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -38,7 +38,8 @@ class KMeansModel private[mllib] (val clusterCenters: Array[Vector]) extends Ser /** Maps given points to their cluster indices. */ def predict(points: RDD[Vector]): RDD[Int] = { val centersWithNorm = clusterCentersWithNorm - points.map(p => KMeans.findClosest(centersWithNorm, new BreezeVectorWithNorm(p))._1) + val bcCentersWithNorm = points.context.broadcast(centersWithNorm) + points.map(p => KMeans.findClosest(bcCentersWithNorm.value, new BreezeVectorWithNorm(p))._1) } /** Maps given points to their cluster indices. */ @@ -51,7 +52,8 @@ class KMeansModel private[mllib] (val clusterCenters: Array[Vector]) extends Ser */ def computeCost(data: RDD[Vector]): Double = { val centersWithNorm = clusterCentersWithNorm - data.map(p => KMeans.pointCost(centersWithNorm, new BreezeVectorWithNorm(p))).sum() + val bcCentersWithNorm = data.context.broadcast(centersWithNorm) + data.map(p => KMeans.pointCost(bcCentersWithNorm.value, new BreezeVectorWithNorm(p))).sum() } private def clusterCentersWithNorm: Iterable[BreezeVectorWithNorm] = diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 7030eeabe400a..9fd760bf78083 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -163,6 +163,7 @@ object GradientDescent extends Logging { // Initialize weights as a column vector var weights = Vectors.dense(initialWeights.toArray) + val n = weights.size /** * For the first iteration, the regVal will be initialized as sum of weight squares @@ -172,12 +173,13 @@ object GradientDescent extends Logging { weights, Vectors.dense(new Array[Double](weights.size)), 0, 1, regParam)._2 for (i <- 1 to numIterations) { + val bcWeights = data.context.broadcast(weights) // Sample a subset (fraction miniBatchFraction) of the total data // compute and sum up the subgradients on this subset (this is one map-reduce) val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42 + i) - .aggregate((BDV.zeros[Double](weights.size), 0.0))( + .aggregate((BDV.zeros[Double](n), 0.0))( seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => - val l = gradient.compute(features, label, weights, Vectors.fromBreeze(grad)) + val l = gradient.compute(features, label, bcWeights.value, Vectors.fromBreeze(grad)) (grad, loss + l) }, combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index 7bbed9c8fdbef..179cd4a3f1625 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -195,13 +195,14 @@ object LBFGS extends Logging { override def calculate(weights: BDV[Double]) = { // Have a local copy to avoid the serialization of CostFun object which is not serializable. - val localData = data val localGradient = gradient + val n = weights.length + val bcWeights = data.context.broadcast(weights) - val (gradientSum, lossSum) = localData.aggregate((BDV.zeros[Double](weights.size), 0.0))( + val (gradientSum, lossSum) = data.aggregate((BDV.zeros[Double](n), 0.0))( seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => val l = localGradient.compute( - features, label, Vectors.fromBreeze(weights), Vectors.fromBreeze(grad)) + features, label, Vectors.fromBreeze(bcWeights.value), Vectors.fromBreeze(grad)) (grad, loss + l) }, combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/DistributionGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/DistributionGenerator.scala new file mode 100644 index 0000000000000..7ecb409c4a91a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/DistributionGenerator.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.random + +import cern.jet.random.Poisson +import cern.jet.random.engine.DRand + +import org.apache.spark.annotation.Experimental +import org.apache.spark.util.random.{XORShiftRandom, Pseudorandom} + +/** + * :: Experimental :: + * Trait for random number generators that generate i.i.d. values from a distribution. + */ +@Experimental +trait DistributionGenerator extends Pseudorandom with Serializable { + + /** + * Returns an i.i.d. sample as a Double from an underlying distribution. + */ + def nextValue(): Double + + /** + * Returns a copy of the DistributionGenerator with a new instance of the rng object used in the + * class when applicable for non-locking concurrent usage. + */ + def copy(): DistributionGenerator +} + +/** + * :: Experimental :: + * Generates i.i.d. samples from U[0.0, 1.0] + */ +@Experimental +class UniformGenerator extends DistributionGenerator { + + // XORShiftRandom for better performance. Thread safety isn't necessary here. + private val random = new XORShiftRandom() + + override def nextValue(): Double = { + random.nextDouble() + } + + override def setSeed(seed: Long) = random.setSeed(seed) + + override def copy(): UniformGenerator = new UniformGenerator() +} + +/** + * :: Experimental :: + * Generates i.i.d. samples from the standard normal distribution. + */ +@Experimental +class StandardNormalGenerator extends DistributionGenerator { + + // XORShiftRandom for better performance. Thread safety isn't necessary here. + private val random = new XORShiftRandom() + + override def nextValue(): Double = { + random.nextGaussian() + } + + override def setSeed(seed: Long) = random.setSeed(seed) + + override def copy(): StandardNormalGenerator = new StandardNormalGenerator() +} + +/** + * :: Experimental :: + * Generates i.i.d. samples from the Poisson distribution with the given mean. + * + * @param mean mean for the Poisson distribution. + */ +@Experimental +class PoissonGenerator(val mean: Double) extends DistributionGenerator { + + private var rng = new Poisson(mean, new DRand) + + override def nextValue(): Double = rng.nextDouble() + + override def setSeed(seed: Long) { + rng = new Poisson(mean, new DRand(seed.toInt)) + } + + override def copy(): PoissonGenerator = new PoissonGenerator(mean) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala new file mode 100644 index 0000000000000..d7ee2d3f46846 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala @@ -0,0 +1,473 @@ +/* + * 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.mllib.random + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.rdd.{RandomVectorRDD, RandomRDD} +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +/** + * :: Experimental :: + * Generator methods for creating RDDs comprised of i.i.d samples from some distribution. + */ +@Experimental +object RandomRDDGenerators { + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the uniform distribution on [0.0, 1.0]. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Double] comprised of i.i.d. samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformRDD(sc: SparkContext, size: Long, numPartitions: Int, seed: Long): RDD[Double] = { + val uniform = new UniformGenerator() + randomRDD(sc, uniform, size, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the uniform distribution on [0.0, 1.0]. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformRDD(sc: SparkContext, size: Long, numPartitions: Int): RDD[Double] = { + uniformRDD(sc, size, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the uniform distribution on [0.0, 1.0]. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformRDD(sc: SparkContext, size: Long): RDD[Double] = { + uniformRDD(sc, size, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the standard normal distribution. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). + */ + @Experimental + def normalRDD(sc: SparkContext, size: Long, numPartitions: Int, seed: Long): RDD[Double] = { + val normal = new StandardNormalGenerator() + randomRDD(sc, normal, size, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the standard normal distribution. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). + */ + @Experimental + def normalRDD(sc: SparkContext, size: Long, numPartitions: Int): RDD[Double] = { + normalRDD(sc, size, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the standard normal distribution. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). + */ + @Experimental + def normalRDD(sc: SparkContext, size: Long): RDD[Double] = { + normalRDD(sc, size, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the Poisson distribution with the input mean. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + */ + @Experimental + def poissonRDD(sc: SparkContext, + mean: Double, + size: Long, + numPartitions: Int, + seed: Long): RDD[Double] = { + val poisson = new PoissonGenerator(mean) + randomRDD(sc, poisson, size, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the Poisson distribution with the input mean. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + */ + @Experimental + def poissonRDD(sc: SparkContext, mean: Double, size: Long, numPartitions: Int): RDD[Double] = { + poissonRDD(sc, mean, size, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the Poisson distribution with the input mean. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param size Size of the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + */ + @Experimental + def poissonRDD(sc: SparkContext, mean: Double, size: Long): RDD[Double] = { + poissonRDD(sc, mean, size, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples produced by the input DistributionGenerator. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Double] comprised of i.i.d. samples produced by generator. + */ + @Experimental + def randomRDD(sc: SparkContext, + generator: DistributionGenerator, + size: Long, + numPartitions: Int, + seed: Long): RDD[Double] = { + new RandomRDD(sc, size, numPartitions, generator, seed) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples produced by the input DistributionGenerator. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Double] comprised of i.i.d. samples produced by generator. + */ + @Experimental + def randomRDD(sc: SparkContext, + generator: DistributionGenerator, + size: Long, + numPartitions: Int): RDD[Double] = { + randomRDD(sc, generator, size, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples produced by the input DistributionGenerator. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param size Size of the RDD. + * @return RDD[Double] comprised of i.i.d. samples produced by generator. + */ + @Experimental + def randomRDD(sc: SparkContext, + generator: DistributionGenerator, + size: Long): RDD[Double] = { + randomRDD(sc, generator, size, sc.defaultParallelism, Utils.random.nextLong) + } + + // TODO Generate RDD[Vector] from multivariate distributions. + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * uniform distribution on [0.0 1.0]. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Vector] with vectors containing i.i.d samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformVectorRDD(sc: SparkContext, + numRows: Long, + numCols: Int, + numPartitions: Int, + seed: Long): RDD[Vector] = { + val uniform = new UniformGenerator() + randomVectorRDD(sc, uniform, numRows, numCols, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * uniform distribution on [0.0 1.0]. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Vector] with vectors containing i.i.d samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformVectorRDD(sc: SparkContext, + numRows: Long, + numCols: Int, + numPartitions: Int): RDD[Vector] = { + uniformVectorRDD(sc, numRows, numCols, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * uniform distribution on [0.0 1.0]. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @return RDD[Vector] with vectors containing i.i.d samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformVectorRDD(sc: SparkContext, numRows: Long, numCols: Int): RDD[Vector] = { + uniformVectorRDD(sc, numRows, numCols, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * standard normal distribution. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Vector] with vectors containing i.i.d samples ~ N(0.0, 1.0). + */ + @Experimental + def normalVectorRDD(sc: SparkContext, + numRows: Long, + numCols: Int, + numPartitions: Int, + seed: Long): RDD[Vector] = { + val uniform = new StandardNormalGenerator() + randomVectorRDD(sc, uniform, numRows, numCols, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * standard normal distribution. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Vector] with vectors containing i.i.d samples ~ N(0.0, 1.0). + */ + @Experimental + def normalVectorRDD(sc: SparkContext, + numRows: Long, + numCols: Int, + numPartitions: Int): RDD[Vector] = { + normalVectorRDD(sc, numRows, numCols, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * standard normal distribution. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @return RDD[Vector] with vectors containing i.i.d samples ~ N(0.0, 1.0). + */ + @Experimental + def normalVectorRDD(sc: SparkContext, numRows: Long, numCols: Int): RDD[Vector] = { + normalVectorRDD(sc, numRows, numCols, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * Poisson distribution with the input mean. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Vector] with vectors containing i.i.d samples ~ Pois(mean). + */ + @Experimental + def poissonVectorRDD(sc: SparkContext, + mean: Double, + numRows: Long, + numCols: Int, + numPartitions: Int, + seed: Long): RDD[Vector] = { + val poisson = new PoissonGenerator(mean) + randomVectorRDD(sc, poisson, numRows, numCols, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * Poisson distribution with the input mean. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Vector] with vectors containing i.i.d samples ~ Pois(mean). + */ + @Experimental + def poissonVectorRDD(sc: SparkContext, + mean: Double, + numRows: Long, + numCols: Int, + numPartitions: Int): RDD[Vector] = { + poissonVectorRDD(sc, mean, numRows, numCols, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * Poisson distribution with the input mean. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @return RDD[Vector] with vectors containing i.i.d samples ~ Pois(mean). + */ + @Experimental + def poissonVectorRDD(sc: SparkContext, + mean: Double, + numRows: Long, + numCols: Int): RDD[Vector] = { + poissonVectorRDD(sc, mean, numRows, numCols, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples produced by the + * input DistributionGenerator. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Vector] with vectors containing i.i.d samples produced by generator. + */ + @Experimental + def randomVectorRDD(sc: SparkContext, + generator: DistributionGenerator, + numRows: Long, + numCols: Int, + numPartitions: Int, + seed: Long): RDD[Vector] = { + new RandomVectorRDD(sc, numRows, numCols, numPartitions, generator, seed) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples produced by the + * input DistributionGenerator. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Vector] with vectors containing i.i.d samples produced by generator. + */ + @Experimental + def randomVectorRDD(sc: SparkContext, + generator: DistributionGenerator, + numRows: Long, + numCols: Int, + numPartitions: Int): RDD[Vector] = { + randomVectorRDD(sc, generator, numRows, numCols, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples produced by the + * input DistributionGenerator. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @return RDD[Vector] with vectors containing i.i.d samples produced by generator. + */ + @Experimental + def randomVectorRDD(sc: SparkContext, + generator: DistributionGenerator, + numRows: Long, + numCols: Int): RDD[Vector] = { + randomVectorRDD(sc, generator, numRows, numCols, + sc.defaultParallelism, Utils.random.nextLong) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala new file mode 100644 index 0000000000000..f13282d07ff92 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala @@ -0,0 +1,118 @@ +/* + * 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.mllib.rdd + +import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.mllib.linalg.{DenseVector, Vector} +import org.apache.spark.mllib.random.DistributionGenerator +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +import scala.util.Random + +private[mllib] class RandomRDDPartition(override val index: Int, + val size: Int, + val generator: DistributionGenerator, + val seed: Long) extends Partition { + + require(size >= 0, "Non-negative partition size required.") +} + +// These two classes are necessary since Range objects in Scala cannot have size > Int.MaxValue +private[mllib] class RandomRDD(@transient sc: SparkContext, + size: Long, + numPartitions: Int, + @transient rng: DistributionGenerator, + @transient seed: Long = Utils.random.nextLong) extends RDD[Double](sc, Nil) { + + require(size > 0, "Positive RDD size required.") + require(numPartitions > 0, "Positive number of partitions required") + require(math.ceil(size.toDouble / numPartitions) <= Int.MaxValue, + "Partition size cannot exceed Int.MaxValue") + + override def compute(splitIn: Partition, context: TaskContext): Iterator[Double] = { + val split = splitIn.asInstanceOf[RandomRDDPartition] + RandomRDD.getPointIterator(split) + } + + override def getPartitions: Array[Partition] = { + RandomRDD.getPartitions(size, numPartitions, rng, seed) + } +} + +private[mllib] class RandomVectorRDD(@transient sc: SparkContext, + size: Long, + vectorSize: Int, + numPartitions: Int, + @transient rng: DistributionGenerator, + @transient seed: Long = Utils.random.nextLong) extends RDD[Vector](sc, Nil) { + + require(size > 0, "Positive RDD size required.") + require(numPartitions > 0, "Positive number of partitions required") + require(vectorSize > 0, "Positive vector size required.") + require(math.ceil(size.toDouble / numPartitions) <= Int.MaxValue, + "Partition size cannot exceed Int.MaxValue") + + override def compute(splitIn: Partition, context: TaskContext): Iterator[Vector] = { + val split = splitIn.asInstanceOf[RandomRDDPartition] + RandomRDD.getVectorIterator(split, vectorSize) + } + + override protected def getPartitions: Array[Partition] = { + RandomRDD.getPartitions(size, numPartitions, rng, seed) + } +} + +private[mllib] object RandomRDD { + + def getPartitions(size: Long, + numPartitions: Int, + rng: DistributionGenerator, + seed: Long): Array[Partition] = { + + val partitions = new Array[RandomRDDPartition](numPartitions) + var i = 0 + var start: Long = 0 + var end: Long = 0 + val random = new Random(seed) + while (i < numPartitions) { + end = ((i + 1) * size) / numPartitions + partitions(i) = new RandomRDDPartition(i, (end - start).toInt, rng, random.nextLong()) + start = end + i += 1 + } + partitions.asInstanceOf[Array[Partition]] + } + + // The RNG has to be reset every time the iterator is requested to guarantee same data + // every time the content of the RDD is examined. + def getPointIterator(partition: RandomRDDPartition): Iterator[Double] = { + val generator = partition.generator.copy() + generator.setSeed(partition.seed) + Array.fill(partition.size)(generator.nextValue()).toIterator + } + + // The RNG has to be reset every time the iterator is requested to guarantee same data + // every time the content of the RDD is examined. + def getVectorIterator(partition: RandomRDDPartition, vectorSize: Int): Iterator[Vector] = { + val generator = partition.generator.copy() + generator.setSeed(partition.seed) + Array.fill(partition.size)(new DenseVector( + (0 until vectorSize).map { _ => generator.nextValue() }.toArray)).toIterator + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index fe41863bce985..54854252d7477 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -56,9 +56,12 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double // A small optimization to avoid serializing the entire model. Only the weightsMatrix // and intercept is needed. val localWeights = weights + val bcWeights = testData.context.broadcast(localWeights) val localIntercept = intercept - - testData.map(v => predictPoint(v, localWeights, localIntercept)) + testData.mapPartitions { iter => + val w = bcWeights.value + iter.map(v => predictPoint(v, w, localIntercept)) + } } /** diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java index faa675b59cd50..862221d48798a 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java @@ -92,8 +92,6 @@ public void runLRUsingStaticMethods() { testRDD.rdd(), 100, 1.0, 1.0); int numAccurate = validatePrediction(validationData, model); - System.out.println(numAccurate); Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); } - } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala index 642843f90204c..d94cfa2fcec81 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala @@ -57,4 +57,12 @@ class PythonMLLibAPISuite extends FunSuite { assert(q.features === p.features) } } + + test("double serialization") { + for (x <- List(123.0, -10.0, 0.0, Double.MaxValue, Double.MinValue)) { + val bytes = py.serializeDouble(x) + val deser = py.deserializeDouble(bytes) + assert(x === deser) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 44b757b6a1fb7..3f6ff859374c7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.Matchers import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} object LogisticRegressionSuite { @@ -126,3 +126,19 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Match validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } } + +class LogisticRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = LogisticRegressionWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index 516895d04222d..06cdd04f5fdae 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -23,7 +23,7 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} object NaiveBayesSuite { @@ -96,3 +96,21 @@ class NaiveBayesSuite extends FunSuite with LocalSparkContext { validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } } + +class NaiveBayesClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 10 + val n = 200000 + val examples = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map { i => + LabeledPoint(random.nextInt(2), Vectors.dense(Array.fill(n)(random.nextDouble()))) + } + } + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = NaiveBayes.train(examples) + val predictions = model.predict(examples.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala index 886c71dde3af7..65e5df58db4c7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala @@ -17,17 +17,16 @@ package org.apache.spark.mllib.classification -import scala.util.Random import scala.collection.JavaConversions._ - -import org.scalatest.FunSuite +import scala.util.Random import org.jblas.DoubleMatrix +import org.scalatest.FunSuite import org.apache.spark.SparkException -import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression._ +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} object SVMSuite { @@ -193,3 +192,19 @@ class SVMSuite extends FunSuite with LocalSparkContext { new SVMWithSGD().setValidateData(false).run(testRDDInvalid) } } + +class SVMClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = SVMWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index 76a3bdf9b11c8..34bc4537a7b3a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -17,14 +17,16 @@ package org.apache.spark.mllib.clustering +import scala.util.Random + import org.scalatest.FunSuite -import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} class KMeansSuite extends FunSuite with LocalSparkContext { - import KMeans.{RANDOM, K_MEANS_PARALLEL} + import org.apache.spark.mllib.clustering.KMeans.{K_MEANS_PARALLEL, RANDOM} test("single cluster") { val data = sc.parallelize(Array( @@ -38,26 +40,26 @@ class KMeansSuite extends FunSuite with LocalSparkContext { // No matter how many runs or iterations we use, we should get one cluster, // centered at the mean of the points - var model = KMeans.train(data, k=1, maxIterations=1) + var model = KMeans.train(data, k = 1, maxIterations = 1) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=2) + model = KMeans.train(data, k = 1, maxIterations = 2) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=5) + model = KMeans.train(data, k = 1, maxIterations = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) assert(model.clusterCenters.head === center) model = KMeans.train( - data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) + data, k = 1, maxIterations = 1, runs = 1, initializationMode = K_MEANS_PARALLEL) assert(model.clusterCenters.head === center) } @@ -100,26 +102,27 @@ class KMeansSuite extends FunSuite with LocalSparkContext { val center = Vectors.dense(1.0, 3.0, 4.0) - var model = KMeans.train(data, k=1, maxIterations=1) + var model = KMeans.train(data, k = 1, maxIterations = 1) assert(model.clusterCenters.size === 1) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=2) + model = KMeans.train(data, k = 1, maxIterations = 2) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=5) + model = KMeans.train(data, k = 1, maxIterations = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, + initializationMode = K_MEANS_PARALLEL) assert(model.clusterCenters.head === center) } @@ -145,25 +148,26 @@ class KMeansSuite extends FunSuite with LocalSparkContext { val center = Vectors.sparse(n, Seq((0, 1.0), (1, 3.0), (2, 4.0))) - var model = KMeans.train(data, k=1, maxIterations=1) + var model = KMeans.train(data, k = 1, maxIterations = 1) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=2) + model = KMeans.train(data, k = 1, maxIterations = 2) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=5) + model = KMeans.train(data, k = 1, maxIterations = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, + initializationMode = K_MEANS_PARALLEL) assert(model.clusterCenters.head === center) data.unpersist() @@ -183,15 +187,15 @@ class KMeansSuite extends FunSuite with LocalSparkContext { // it will make at least five passes, and it will give non-zero probability to each // unselected point as long as it hasn't yet selected all of them - var model = KMeans.train(rdd, k=5, maxIterations=1) + var model = KMeans.train(rdd, k = 5, maxIterations = 1) assert(Set(model.clusterCenters: _*) === Set(points: _*)) // Iterations of Lloyd's should not change the answer either - model = KMeans.train(rdd, k=5, maxIterations=10) + model = KMeans.train(rdd, k = 5, maxIterations = 10) assert(Set(model.clusterCenters: _*) === Set(points: _*)) // Neither should more runs - model = KMeans.train(rdd, k=5, maxIterations=10, runs=5) + model = KMeans.train(rdd, k = 5, maxIterations = 10, runs = 5) assert(Set(model.clusterCenters: _*) === Set(points: _*)) } @@ -220,3 +224,22 @@ class KMeansSuite extends FunSuite with LocalSparkContext { } } } + +class KMeansClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => Vectors.dense(Array.fill(n)(random.nextDouble))) + }.cache() + for (initMode <- Seq(KMeans.RANDOM, KMeans.K_MEANS_PARALLEL)) { + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = KMeans.train(points, 2, 2, 1, initMode) + val predictions = model.predict(points).collect() + val cost = model.computeCost(points) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index a961f89456a18..325b817980f68 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -17,12 +17,13 @@ package org.apache.spark.mllib.linalg.distributed -import org.scalatest.FunSuite +import scala.util.Random import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, norm => brzNorm, svd => brzSvd} +import org.scalatest.FunSuite -import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.linalg.{Matrices, Vectors, Vector} +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} class RowMatrixSuite extends FunSuite with LocalSparkContext { @@ -193,3 +194,27 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { } } } + +class RowMatrixClusterSuite extends FunSuite with LocalClusterSparkContext { + + var mat: RowMatrix = _ + + override def beforeAll() { + super.beforeAll() + val m = 4 + val n = 200000 + val rows = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => Vectors.dense(Array.fill(n)(random.nextDouble()))) + } + mat = new RowMatrix(rows) + } + + test("task size should be small in svd") { + val svd = mat.computeSVD(1, computeU = true) + } + + test("task size should be small in summarize") { + val summary = mat.computeColumnSummaryStatistics() + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala index 951b4f7c6e6f4..dfb2eb7f0d14e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala @@ -17,15 +17,14 @@ package org.apache.spark.mllib.optimization -import scala.util.Random import scala.collection.JavaConversions._ +import scala.util.Random -import org.scalatest.FunSuite -import org.scalatest.Matchers +import org.scalatest.{FunSuite, Matchers} -import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression._ +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} object GradientDescentSuite { @@ -46,7 +45,7 @@ object GradientDescentSuite { val rnd = new Random(seed) val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - val unifRand = new scala.util.Random(45) + val unifRand = new Random(45) val rLogis = (0 until nPoints).map { i => val u = unifRand.nextDouble() math.log(u) - math.log(1.0-u) @@ -144,3 +143,26 @@ class GradientDescentSuite extends FunSuite with LocalSparkContext with Matchers "should be initialWeightsWithIntercept.") } } + +class GradientDescentClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => (1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val (weights, loss) = GradientDescent.runMiniBatchSGD( + points, + new LogisticGradient, + new SquaredL2Updater, + 0.1, + 2, + 1.0, + 1.0, + Vectors.dense(new Array[Double](n))) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala index fe7a9033cd5f4..ff414742e8393 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala @@ -17,12 +17,13 @@ package org.apache.spark.mllib.optimization -import org.scalatest.FunSuite -import org.scalatest.Matchers +import scala.util.Random + +import org.scalatest.{FunSuite, Matchers} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { @@ -230,3 +231,24 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { "The weight differences between LBFGS and GD should be within 2%.") } } + +class LBFGSClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small") { + val m = 10 + val n = 200000 + val examples = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => (1.0, Vectors.dense(Array.fill(n)(random.nextDouble)))) + }.cache() + val lbfgs = new LBFGS(new LogisticGradient, new SquaredL2Updater) + .setNumCorrections(1) + .setConvergenceTol(1e-12) + .setMaxNumIterations(1) + .setRegParam(1.0) + val random = new Random(0) + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val weights = lbfgs.optimize(examples, Vectors.dense(Array.fill(n)(random.nextDouble))) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/DistributionGeneratorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/DistributionGeneratorSuite.scala new file mode 100644 index 0000000000000..974dec4c0b5ee --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/DistributionGeneratorSuite.scala @@ -0,0 +1,90 @@ +/* + * 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.mllib.random + +import org.scalatest.FunSuite + +import org.apache.spark.util.StatCounter + +// TODO update tests to use TestingUtils for floating point comparison after PR 1367 is merged +class DistributionGeneratorSuite extends FunSuite { + + def apiChecks(gen: DistributionGenerator) { + + // resetting seed should generate the same sequence of random numbers + gen.setSeed(42L) + val array1 = (0 until 1000).map(_ => gen.nextValue()) + gen.setSeed(42L) + val array2 = (0 until 1000).map(_ => gen.nextValue()) + assert(array1.equals(array2)) + + // newInstance should contain a difference instance of the rng + // i.e. setting difference seeds for difference instances produces different sequences of + // random numbers. + val gen2 = gen.copy() + gen.setSeed(0L) + val array3 = (0 until 1000).map(_ => gen.nextValue()) + gen2.setSeed(1L) + val array4 = (0 until 1000).map(_ => gen2.nextValue()) + // Compare arrays instead of elements since individual elements can coincide by chance but the + // sequences should differ given two different seeds. + assert(!array3.equals(array4)) + + // test that setting the same seed in the copied instance produces the same sequence of numbers + gen.setSeed(0L) + val array5 = (0 until 1000).map(_ => gen.nextValue()) + gen2.setSeed(0L) + val array6 = (0 until 1000).map(_ => gen2.nextValue()) + assert(array5.equals(array6)) + } + + def distributionChecks(gen: DistributionGenerator, + mean: Double = 0.0, + stddev: Double = 1.0, + epsilon: Double = 0.01) { + for (seed <- 0 until 5) { + gen.setSeed(seed.toLong) + val sample = (0 until 100000).map { _ => gen.nextValue()} + val stats = new StatCounter(sample) + assert(math.abs(stats.mean - mean) < epsilon) + assert(math.abs(stats.stdev - stddev) < epsilon) + } + } + + test("UniformGenerator") { + val uniform = new UniformGenerator() + apiChecks(uniform) + // Stddev of uniform distribution = (ub - lb) / math.sqrt(12) + distributionChecks(uniform, 0.5, 1 / math.sqrt(12)) + } + + test("StandardNormalGenerator") { + val normal = new StandardNormalGenerator() + apiChecks(normal) + distributionChecks(normal, 0.0, 1.0) + } + + test("PoissonGenerator") { + // mean = 0.0 will not pass the API checks since 0.0 is always deterministically produced. + for (mean <- List(1.0, 5.0, 100.0)) { + val poisson = new PoissonGenerator(mean) + apiChecks(poisson) + distributionChecks(poisson, mean, math.sqrt(mean), 0.1) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala new file mode 100644 index 0000000000000..6aa4f803df0f7 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala @@ -0,0 +1,158 @@ +/* + * 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.mllib.random + +import scala.collection.mutable.ArrayBuffer + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.rdd.{RandomRDDPartition, RandomRDD} +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.util.StatCounter + +/* + * Note: avoid including APIs that do not set the seed for the RNG in unit tests + * in order to guarantee deterministic behavior. + * + * TODO update tests to use TestingUtils for floating point comparison after PR 1367 is merged + */ +class RandomRDDGeneratorsSuite extends FunSuite with LocalSparkContext with Serializable { + + def testGeneratedRDD(rdd: RDD[Double], + expectedSize: Long, + expectedNumPartitions: Int, + expectedMean: Double, + expectedStddev: Double, + epsilon: Double = 0.01) { + val stats = rdd.stats() + assert(expectedSize === stats.count) + assert(expectedNumPartitions === rdd.partitions.size) + assert(math.abs(stats.mean - expectedMean) < epsilon) + assert(math.abs(stats.stdev - expectedStddev) < epsilon) + } + + // assume test RDDs are small + def testGeneratedVectorRDD(rdd: RDD[Vector], + expectedRows: Long, + expectedColumns: Int, + expectedNumPartitions: Int, + expectedMean: Double, + expectedStddev: Double, + epsilon: Double = 0.01) { + assert(expectedNumPartitions === rdd.partitions.size) + val values = new ArrayBuffer[Double]() + rdd.collect.foreach { vector => { + assert(vector.size === expectedColumns) + values ++= vector.toArray + }} + assert(expectedRows === values.size / expectedColumns) + val stats = new StatCounter(values) + assert(math.abs(stats.mean - expectedMean) < epsilon) + assert(math.abs(stats.stdev - expectedStddev) < epsilon) + } + + test("RandomRDD sizes") { + + // some cases where size % numParts != 0 to test getPartitions behaves correctly + for ((size, numPartitions) <- List((10000, 6), (12345, 1), (1000, 101))) { + val rdd = new RandomRDD(sc, size, numPartitions, new UniformGenerator, 0L) + assert(rdd.count() === size) + assert(rdd.partitions.size === numPartitions) + + // check that partition sizes are balanced + val partSizes = rdd.partitions.map(p => p.asInstanceOf[RandomRDDPartition].size.toDouble) + val partStats = new StatCounter(partSizes) + assert(partStats.max - partStats.min <= 1) + } + + // size > Int.MaxValue + val size = Int.MaxValue.toLong * 100L + val numPartitions = 101 + val rdd = new RandomRDD(sc, size, numPartitions, new UniformGenerator, 0L) + assert(rdd.partitions.size === numPartitions) + val count = rdd.partitions.foldLeft(0L) { (count, part) => + count + part.asInstanceOf[RandomRDDPartition].size + } + assert(count === size) + + // size needs to be positive + intercept[IllegalArgumentException] { new RandomRDD(sc, 0, 10, new UniformGenerator, 0L) } + + // numPartitions needs to be positive + intercept[IllegalArgumentException] { new RandomRDD(sc, 100, 0, new UniformGenerator, 0L) } + + // partition size needs to be <= Int.MaxValue + intercept[IllegalArgumentException] { + new RandomRDD(sc, Int.MaxValue.toLong * 100L, 99, new UniformGenerator, 0L) + } + } + + test("randomRDD for different distributions") { + val size = 100000L + val numPartitions = 10 + val poissonMean = 100.0 + + for (seed <- 0 until 5) { + val uniform = RandomRDDGenerators.uniformRDD(sc, size, numPartitions, seed) + testGeneratedRDD(uniform, size, numPartitions, 0.5, 1 / math.sqrt(12)) + + val normal = RandomRDDGenerators.normalRDD(sc, size, numPartitions, seed) + testGeneratedRDD(normal, size, numPartitions, 0.0, 1.0) + + val poisson = RandomRDDGenerators.poissonRDD(sc, poissonMean, size, numPartitions, seed) + testGeneratedRDD(poisson, size, numPartitions, poissonMean, math.sqrt(poissonMean), 0.1) + } + + // mock distribution to check that partitions have unique seeds + val random = RandomRDDGenerators.randomRDD(sc, new MockDistro(), 1000L, 1000, 0L) + assert(random.collect.size === random.collect.distinct.size) + } + + test("randomVectorRDD for different distributions") { + val rows = 1000L + val cols = 100 + val parts = 10 + val poissonMean = 100.0 + + for (seed <- 0 until 5) { + val uniform = RandomRDDGenerators.uniformVectorRDD(sc, rows, cols, parts, seed) + testGeneratedVectorRDD(uniform, rows, cols, parts, 0.5, 1 / math.sqrt(12)) + + val normal = RandomRDDGenerators.normalVectorRDD(sc, rows, cols, parts, seed) + testGeneratedVectorRDD(normal, rows, cols, parts, 0.0, 1.0) + + val poisson = RandomRDDGenerators.poissonVectorRDD(sc, poissonMean, rows, cols, parts, seed) + testGeneratedVectorRDD(poisson, rows, cols, parts, poissonMean, math.sqrt(poissonMean), 0.1) + } + } +} + +private[random] class MockDistro extends DistributionGenerator { + + var seed = 0L + + // This allows us to check that each partition has a different seed + override def nextValue(): Double = seed.toDouble + + override def setSeed(seed: Long) = this.seed = seed + + override def copy(): MockDistro = new MockDistro +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala index bfa42959c8ead..7aa96421aed87 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala @@ -17,10 +17,13 @@ package org.apache.spark.mllib.regression +import scala.util.Random + import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, + LocalSparkContext} class LassoSuite extends FunSuite with LocalSparkContext { @@ -113,3 +116,19 @@ class LassoSuite extends FunSuite with LocalSparkContext { validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } } + +class LassoClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = LassoWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala index 7aaad7d7a3e39..4f89112b650c5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala @@ -17,10 +17,13 @@ package org.apache.spark.mllib.regression +import scala.util.Random + import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, + LocalSparkContext} class LinearRegressionSuite extends FunSuite with LocalSparkContext { @@ -122,3 +125,19 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { sparseValidationData.map(row => model.predict(row.features)), sparseValidationData) } } + +class LinearRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = LinearRegressionWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala index 67768e17fbe6d..727bbd051ff15 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala @@ -17,11 +17,14 @@ package org.apache.spark.mllib.regression -import org.scalatest.FunSuite +import scala.util.Random import org.jblas.DoubleMatrix +import org.scalatest.FunSuite -import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, + LocalSparkContext} class RidgeRegressionSuite extends FunSuite with LocalSparkContext { @@ -73,3 +76,19 @@ class RidgeRegressionSuite extends FunSuite with LocalSparkContext { "ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")") } } + +class RidgeRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = RidgeRegressionWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala new file mode 100644 index 0000000000000..5e9101cdd3804 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala @@ -0,0 +1,42 @@ +/* + * 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.mllib.util + +import org.scalatest.{Suite, BeforeAndAfterAll} + +import org.apache.spark.{SparkConf, SparkContext} + +trait LocalClusterSparkContext extends BeforeAndAfterAll { self: Suite => + @transient var sc: SparkContext = _ + + override def beforeAll() { + val conf = new SparkConf() + .setMaster("local-cluster[2, 1, 512]") + .setAppName("test-cluster") + .set("spark.akka.frameSize", "1") // set to 1MB to detect direct serialization of data + sc = new SparkContext(conf) + super.beforeAll() + } + + override def afterAll() { + if (sc != null) { + sc.stop() + } + super.afterAll() + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala index 0d4868f3d9e42..7857d9e5ee5c4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala @@ -20,13 +20,16 @@ package org.apache.spark.mllib.util import org.scalatest.Suite import org.scalatest.BeforeAndAfterAll -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} trait LocalSparkContext extends BeforeAndAfterAll { self: Suite => @transient var sc: SparkContext = _ override def beforeAll() { - sc = new SparkContext("local", "test") + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + sc = new SparkContext(conf) super.beforeAll() } diff --git a/pom.xml b/pom.xml index 96a0c60d24de6..20cc248772aac 100644 --- a/pom.xml +++ b/pom.xml @@ -95,7 +95,6 @@ sql/catalyst sql/core sql/hive - sql/hive-thriftserver repl assembly external/twitter @@ -254,9 +253,9 @@ 3.3.2 - commons-codec - commons-codec - 1.5 + commons-codec + commons-codec + 1.5 com.google.code.findbugs @@ -959,6 +958,30 @@ org.apache.maven.plugins maven-source-plugin + + org.scalastyle + scalastyle-maven-plugin + 0.4.0 + + false + true + false + false + ${basedir}/src/main/scala + ${basedir}/src/test/scala + scalastyle-config.xml + scalastyle-output.xml + UTF-8 + + + + package + + check + + + + diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index e9220db6b1f9a..5ff88f0dd1cac 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -31,7 +31,6 @@ import com.typesafe.tools.mima.core._ * MimaBuild.excludeSparkClass("graphx.util.collection.GraphXPrimitiveKeyOpenHashMap") */ object MimaExcludes { - def excludes(version: String) = version match { case v if v.startsWith("1.1") => @@ -62,6 +61,15 @@ object MimaExcludes { ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.storage.MemoryStore.Entry") ) ++ + Seq( + // Renamed putValues -> putArray + putIterator + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.storage.MemoryStore.putValues"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.storage.DiskStore.putValues"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.storage.TachyonStore.putValues") + ) ++ Seq( ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.FlumeReceiver.this") ) ++ diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 21dcb40e9a22f..ae985fe549c85 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -30,13 +30,12 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile - val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, spark, sql, - streaming, streamingFlumeSink, streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, + val allProjects@Seq(bagel, catalyst, core, graphx, hive, mllib, repl, spark, sql, streaming, + streamingFlumeSink, streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, streamingZeromq) = - Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", - "spark", "sql", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", - "streaming-mqtt", "streaming-twitter", "streaming-zeromq"). - map(ProjectRef(buildLocation, _)) + Seq("bagel", "catalyst", "core", "graphx", "hive", "mllib", "repl", "spark", "sql", + "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", "streaming-mqtt", + "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, sparkGangliaLgpl) = Seq("yarn", "yarn-stable", "yarn-alpha", "java8-tests", "ganglia-lgpl") @@ -102,7 +101,7 @@ object SparkBuild extends PomBuild { Properties.envOrNone("SBT_MAVEN_PROPERTIES") match { case Some(v) => v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.split("=")).foreach(x => System.setProperty(x(0), x(1))) - case _ => + case _ => } override val userPropertiesMap = System.getProperties.toMap @@ -160,7 +159,7 @@ object SparkBuild extends PomBuild { /* Enable Mima for all projects except spark, hive, catalyst, sql and repl */ // TODO: Add Sql to mima checks - allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl).contains(x)). + allProjects.filterNot(y => Seq(spark, sql, hive, catalyst, repl).exists(x => x == y)). foreach (x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)) /* Enable Assembly for all assembly projects */ diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 024fb881877c9..e8ac9895cf54a 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -37,6 +37,15 @@ from py4j.java_collections import ListConverter +# These are special default configs for PySpark, they will overwrite +# the default ones for Spark if they are not configured by user. +DEFAULT_CONFIGS = { + "spark.serializer": "org.apache.spark.serializer.KryoSerializer", + "spark.serializer.objectStreamReset": 100, + "spark.rdd.compress": True, +} + + class SparkContext(object): """ Main entry point for Spark functionality. A SparkContext represents the @@ -101,7 +110,7 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, else: self.serializer = BatchedSerializer(self._unbatched_serializer, batchSize) - self._conf.setIfMissing("spark.rdd.compress", "true") + # Set any parameters passed directly to us on the conf if master: self._conf.setMaster(master) @@ -112,6 +121,8 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, if environment: for key, value in environment.iteritems(): self._conf.setExecutorEnv(key, value) + for key, value in DEFAULT_CONFIGS.items(): + self._conf.setIfMissing(key, value) # Check that we have at least the required parameters if not self._conf.contains("spark.master"): @@ -216,6 +227,13 @@ def setSystemProperty(cls, key, value): SparkContext._ensure_initialized() SparkContext._jvm.java.lang.System.setProperty(key, value) + @property + def version(self): + """ + The version of Spark on which this application is running. + """ + return self._jsc.version() + @property def defaultParallelism(self): """ diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index 43b491a9716fc..8e3ad6b783b6c 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -72,9 +72,9 @@ # Python interpreter must agree on what endian the machine is. -DENSE_VECTOR_MAGIC = 1 +DENSE_VECTOR_MAGIC = 1 SPARSE_VECTOR_MAGIC = 2 -DENSE_MATRIX_MAGIC = 3 +DENSE_MATRIX_MAGIC = 3 LABELED_POINT_MAGIC = 4 @@ -97,8 +97,28 @@ def _deserialize_numpy_array(shape, ba, offset, dtype=float64): return ar.copy() +def _serialize_double(d): + """ + Serialize a double (float or numpy.float64) into a mutually understood format. + """ + if type(d) == float or type(d) == float64: + d = float64(d) + ba = bytearray(8) + _copyto(d, buffer=ba, offset=0, shape=[1], dtype=float64) + return ba + else: + raise TypeError("_serialize_double called on non-float input") + + def _serialize_double_vector(v): - """Serialize a double vector into a mutually understood format. + """ + Serialize a double vector into a mutually understood format. + + Note: we currently do not use a magic byte for double for storage + efficiency. This should be reconsidered when we add Ser/De for other + 8-byte types (e.g. Long), for safety. The corresponding deserializer, + _deserialize_double, needs to be modified as well if the serialization + scheme changes. >>> x = array([1,2,3]) >>> y = _deserialize_double_vector(_serialize_double_vector(x)) @@ -148,6 +168,28 @@ def _serialize_sparse_vector(v): return ba +def _deserialize_double(ba, offset=0): + """Deserialize a double from a mutually understood format. + + >>> import sys + >>> _deserialize_double(_serialize_double(123.0)) == 123.0 + True + >>> _deserialize_double(_serialize_double(float64(0.0))) == 0.0 + True + >>> x = sys.float_info.max + >>> _deserialize_double(_serialize_double(sys.float_info.max)) == x + True + >>> y = float64(sys.float_info.max) + >>> _deserialize_double(_serialize_double(sys.float_info.max)) == y + True + """ + if type(ba) != bytearray: + raise TypeError("_deserialize_double called on a %s; wanted bytearray" % type(ba)) + if len(ba) - offset != 8: + raise TypeError("_deserialize_double called on a %d-byte array; wanted 8 bytes." % nb) + return struct.unpack("d", ba[offset:])[0] + + def _deserialize_double_vector(ba, offset=0): """Deserialize a double vector from a mutually understood format. diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 113a082e16721..b84d976114f0d 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1687,7 +1687,6 @@ def _jrdd(self): [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], self.ctx._gateway._gateway_client) self.ctx._pickled_broadcast_vars.clear() - class_tag = self._prev_jrdd.classTag() env = MapConverter().convert(self.ctx.environment, self.ctx._gateway._gateway_client) includes = ListConverter().convert(self.ctx._python_includes, @@ -1696,8 +1695,7 @@ def _jrdd(self): bytearray(pickled_command), env, includes, self.preservesPartitioning, self.ctx.pythonExec, - broadcast_vars, self.ctx._javaAccumulator, - class_tag) + broadcast_vars, self.ctx._javaAccumulator) self._jrdd_val = python_rdd.asJavaRDD() return self._jrdd_val diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index a92abbf371f18..8ba51461d106d 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -226,6 +226,15 @@ def test_transforming_cartesian_result(self): cart = rdd1.cartesian(rdd2) result = cart.map(lambda (x, y): x + y).collect() + def test_transforming_pickle_file(self): + # Regression test for SPARK-2601 + data = self.sc.parallelize(["Hello", "World!"]) + tempFile = tempfile.NamedTemporaryFile(delete=True) + tempFile.close() + data.saveAsPickleFile(tempFile.name) + pickled_file = self.sc.pickleFile(tempFile.name) + pickled_file.map(lambda x: x).collect() + def test_cartesian_on_textfile(self): # Regression test for path = os.path.join(SPARK_HOME, "python/test_support/hello.txt") diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh deleted file mode 100755 index 8398e6f19b511..0000000000000 --- a/sbin/start-thriftserver.sh +++ /dev/null @@ -1,36 +0,0 @@ -#!/usr/bin/env bash - -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# -# Shell script for starting the Spark SQL Thrift server - -# Enter posix mode for bash -set -o posix - -# Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" - -if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - echo "Usage: ./sbin/start-thriftserver [options]" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit 0 -fi - -CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" -exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal $@ diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 531bfddbf237b..6decde3fcd62d 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -32,7 +32,7 @@ Spark Project Catalyst http://spark.apache.org/ - catalyst + catalyst diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 67a8ce9b88c3f..47c7ad076ad07 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -50,6 +50,7 @@ trait HiveTypeCoercion { StringToIntegralCasts :: FunctionArgumentConversion :: CastNulls :: + Division :: Nil /** @@ -317,6 +318,23 @@ trait HiveTypeCoercion { } } + /** + * Hive only performs integral division with the DIV operator. The arguments to / are always + * converted to fractional types. + */ + object Division extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + + // Decimal and Double remain the same + case d: Divide if d.dataType == DoubleType => d + case d: Divide if d.dataType == DecimalType => d + + case Divide(l, r) => Divide(Cast(l, DoubleType), Cast(r, DoubleType)) + } + } + /** * Ensures that NullType gets casted to some other types under certain circumstances. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index a357c6ffb8977..1d5f033f0d274 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -43,7 +43,8 @@ case class NativeCommand(cmd: String) extends Command { */ case class SetCommand(key: Option[String], value: Option[String]) extends Command { override def output = Seq( - BoundReference(1, AttributeReference("", StringType, nullable = false)())) + BoundReference(0, AttributeReference("key", StringType, nullable = false)()), + BoundReference(1, AttributeReference("value", StringType, nullable = false)())) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index d607eed1bea89..0a27cce337482 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -83,7 +83,7 @@ class ConstantFoldingSuite extends PlanTest { Literal(10) as Symbol("2*3+4"), Literal(14) as Symbol("2*(3+4)")) .where(Literal(true)) - .groupBy(Literal(3))(Literal(3) as Symbol("9/3")) + .groupBy(Literal(3.0))(Literal(3.0) as Symbol("9/3")) .analyze comparePlans(optimized, correctAnswer) diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3a038a2db6173..c309c43804d97 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -32,7 +32,7 @@ Spark Project SQL http://spark.apache.org/ - sql + sql diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 41920c00b5a2c..2b787e14f3f15 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -30,13 +30,12 @@ import scala.collection.JavaConverters._ * SQLConf is thread-safe (internally synchronized so safe to be used in multiple threads). */ trait SQLConf { - import SQLConf._ /** ************************ Spark SQL Params/Hints ******************* */ // TODO: refactor so that these hints accessors don't pollute the name space of SQLContext? /** Number of partitions to use for shuffle operators. */ - private[spark] def numShufflePartitions: Int = get(SHUFFLE_PARTITIONS, "200").toInt + private[spark] def numShufflePartitions: Int = get("spark.sql.shuffle.partitions", "200").toInt /** * Upper bound on the sizes (in bytes) of the tables qualified for the auto conversion to @@ -44,10 +43,11 @@ trait SQLConf { * effectively disables auto conversion. * Hive setting: hive.auto.convert.join.noconditionaltask.size. */ - private[spark] def autoConvertJoinSize: Int = get(AUTO_CONVERT_JOIN_SIZE, "10000").toInt + private[spark] def autoConvertJoinSize: Int = + get("spark.sql.auto.convert.join.size", "10000").toInt /** A comma-separated list of table names marked to be broadcasted during joins. */ - private[spark] def joinBroadcastTables: String = get(JOIN_BROADCAST_TABLES, "") + private[spark] def joinBroadcastTables: String = get("spark.sql.join.broadcastTables", "") /** ********************** SQLConf functionality methods ************ */ @@ -61,7 +61,7 @@ trait SQLConf { def set(key: String, value: String): Unit = { require(key != null, "key cannot be null") - require(value != null, s"value cannot be null for $key") + require(value != null, s"value cannot be null for ${key}") settings.put(key, value) } @@ -90,13 +90,3 @@ trait SQLConf { } } - -object SQLConf { - val AUTO_CONVERT_JOIN_SIZE = "spark.sql.auto.convert.join.size" - val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" - val JOIN_BROADCAST_TABLES = "spark.sql.join.broadcastTables" - - object Deprecated { - val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 9293239131d52..98d2f89c8ae71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -17,13 +17,12 @@ package org.apache.spark.sql.execution -import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.{Row, SQLConf, SQLContext} +import org.apache.spark.sql.{Row, SQLContext} trait Command { /** @@ -45,53 +44,28 @@ trait Command { case class SetCommand( key: Option[String], value: Option[String], output: Seq[Attribute])( @transient context: SQLContext) - extends LeafNode with Command with Logging { + extends LeafNode with Command { - override protected[sql] lazy val sideEffectResult: Seq[String] = (key, value) match { + override protected[sql] lazy val sideEffectResult: Seq[(String, String)] = (key, value) match { // Set value for key k. case (Some(k), Some(v)) => - if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { - logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + - s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") - context.set(SQLConf.SHUFFLE_PARTITIONS, v) - Array(s"${SQLConf.SHUFFLE_PARTITIONS}=$v") - } else { - context.set(k, v) - Array(s"$k=$v") - } + context.set(k, v) + Array(k -> v) // Query the value bound to key k. case (Some(k), _) => - // TODO (lian) This is just a workaround to make the Simba ODBC driver work. - // Should remove this once we get the ODBC driver updated. - if (k == "-v") { - val hiveJars = Seq( - "hive-exec-0.12.0.jar", - "hive-service-0.12.0.jar", - "hive-common-0.12.0.jar", - "hive-hwi-0.12.0.jar", - "hive-0.12.0.jar").mkString(":") - - Array( - "system:java.class.path=" + hiveJars, - "system:sun.java.command=shark.SharkServer2") - } - else { - Array(s"$k=${context.getOption(k).getOrElse("")}") - } + Array(k -> context.getOption(k).getOrElse("")) // Query all key-value pairs that are set in the SQLConf of the context. case (None, None) => - context.getAll.map { case (k, v) => - s"$k=$v" - } + context.getAll case _ => throw new IllegalArgumentException() } def execute(): RDD[Row] = { - val rows = sideEffectResult.map { line => new GenericRow(Array[Any](line)) } + val rows = sideEffectResult.map { case (k, v) => new GenericRow(Array[Any](k, v)) } context.sparkContext.parallelize(rows, 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala index 1a58d73d9e7f4..08293f7f0ca30 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala @@ -54,10 +54,10 @@ class SQLConfSuite extends QueryTest { assert(get(testKey, testVal + "_") == testVal) assert(TestSQLContext.get(testKey, testVal + "_") == testVal) - sql("set some.property=20") - assert(get("some.property", "0") == "20") - sql("set some.property = 40") - assert(get("some.property", "0") == "40") + sql("set mapred.reduce.tasks=20") + assert(get("mapred.reduce.tasks", "0") == "20") + sql("set mapred.reduce.tasks = 40") + assert(get("mapred.reduce.tasks", "0") == "40") val key = "spark.sql.key" val vs = "val0,val_1,val2.3,my_table" @@ -70,9 +70,4 @@ class SQLConfSuite extends QueryTest { clear() } - test("deprecated property") { - clear() - sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") - assert(get(SQLConf.SHUFFLE_PARTITIONS) == "10") - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index de9e8aa4f62ed..6736189c96d4b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -424,25 +424,25 @@ class SQLQuerySuite extends QueryTest { sql(s"SET $testKey=$testVal") checkAnswer( sql("SET"), - Seq(Seq(s"$testKey=$testVal")) + Seq(Seq(testKey, testVal)) ) sql(s"SET ${testKey + testKey}=${testVal + testVal}") checkAnswer( sql("set"), Seq( - Seq(s"$testKey=$testVal"), - Seq(s"${testKey + testKey}=${testVal + testVal}")) + Seq(testKey, testVal), + Seq(testKey + testKey, testVal + testVal)) ) // "set key" checkAnswer( sql(s"SET $testKey"), - Seq(Seq(s"$testKey=$testVal")) + Seq(Seq(testKey, testVal)) ) checkAnswer( sql(s"SET $nonexistentKey"), - Seq(Seq(s"$nonexistentKey=")) + Seq(Seq(nonexistentKey, "")) ) clear() } diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml deleted file mode 100644 index 7fac90fdc596d..0000000000000 --- a/sql/hive-thriftserver/pom.xml +++ /dev/null @@ -1,82 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent - 1.1.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-hive-thriftserver_2.10 - jar - Spark Project Hive - http://spark.apache.org/ - - hive-thriftserver - - - - - org.apache.spark - spark-hive_${scala.binary.version} - ${project.version} - - - org.spark-project.hive - hive-cli - ${hive.version} - - - org.spark-project.hive - hive-jdbc - ${hive.version} - - - org.spark-project.hive - hive-beeline - ${hive.version} - - - org.scalatest - scalatest_${scala.binary.version} - test - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - - org.apache.maven.plugins - maven-deploy-plugin - - true - - - - - diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala deleted file mode 100644 index ddbc2a79fb512..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ /dev/null @@ -1,97 +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.sql.hive.thriftserver - -import scala.collection.JavaConversions._ - -import org.apache.commons.logging.LogFactory -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService -import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} - -import org.apache.spark.sql.Logging -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ - -/** - * The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a - * `HiveThriftServer2` thrift server. - */ -private[hive] object HiveThriftServer2 extends Logging { - var LOG = LogFactory.getLog(classOf[HiveServer2]) - - def main(args: Array[String]) { - val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") - - if (!optionsProcessor.process(args)) { - logger.warn("Error starting HiveThriftServer2 with given arguments") - System.exit(-1) - } - - val ss = new SessionState(new HiveConf(classOf[SessionState])) - - // Set all properties specified via command line. - val hiveConf: HiveConf = ss.getConf - hiveConf.getAllProperties.toSeq.sortBy(_._1).foreach { case (k, v) => - logger.debug(s"HiveConf var: $k=$v") - } - - SessionState.start(ss) - - logger.info("Starting SparkContext") - SparkSQLEnv.init() - SessionState.start(ss) - - Runtime.getRuntime.addShutdownHook( - new Thread() { - override def run() { - SparkSQLEnv.sparkContext.stop() - } - } - ) - - try { - val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) - server.init(hiveConf) - server.start() - logger.info("HiveThriftServer2 started") - } catch { - case e: Exception => - logger.error("Error starting HiveThriftServer2", e) - System.exit(-1) - } - } -} - -private[hive] class HiveThriftServer2(hiveContext: HiveContext) - extends HiveServer2 - with ReflectedCompositeService { - - override def init(hiveConf: HiveConf) { - val sparkSqlCliService = new SparkSQLCLIService(hiveContext) - setSuperField(this, "cliService", sparkSqlCliService) - addService(sparkSqlCliService) - - val thriftCliService = new ThriftBinaryCLIService(sparkSqlCliService) - setSuperField(this, "thriftCLIService", thriftCliService) - addService(thriftCliService) - - initCompositeService(hiveConf) - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala deleted file mode 100644 index 599294dfbb7d7..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala +++ /dev/null @@ -1,58 +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.sql.hive.thriftserver - -private[hive] object ReflectionUtils { - def setSuperField(obj : Object, fieldName: String, fieldValue: Object) { - setAncestorField(obj, 1, fieldName, fieldValue) - } - - def setAncestorField(obj: AnyRef, level: Int, fieldName: String, fieldValue: AnyRef) { - val ancestor = Iterator.iterate[Class[_]](obj.getClass)(_.getSuperclass).drop(level).next() - val field = ancestor.getDeclaredField(fieldName) - field.setAccessible(true) - field.set(obj, fieldValue) - } - - def getSuperField[T](obj: AnyRef, fieldName: String): T = { - getAncestorField[T](obj, 1, fieldName) - } - - def getAncestorField[T](clazz: Object, level: Int, fieldName: String): T = { - val ancestor = Iterator.iterate[Class[_]](clazz.getClass)(_.getSuperclass).drop(level).next() - val field = ancestor.getDeclaredField(fieldName) - field.setAccessible(true) - field.get(clazz).asInstanceOf[T] - } - - def invokeStatic(clazz: Class[_], methodName: String, args: (Class[_], AnyRef)*): AnyRef = { - invoke(clazz, null, methodName, args: _*) - } - - def invoke( - clazz: Class[_], - obj: AnyRef, - methodName: String, - args: (Class[_], AnyRef)*): AnyRef = { - - val (types, values) = args.unzip - val method = clazz.getDeclaredMethod(methodName, types: _*) - method.setAccessible(true) - method.invoke(obj, values.toSeq: _*) - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala deleted file mode 100755 index 27268ecb923e9..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ /dev/null @@ -1,344 +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.sql.hive.thriftserver - -import scala.collection.JavaConversions._ - -import java.io._ -import java.util.{ArrayList => JArrayList} - -import jline.{ConsoleReader, History} -import org.apache.commons.lang.StringUtils -import org.apache.commons.logging.LogFactory -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor} -import org.apache.hadoop.hive.common.LogUtils.LogInitializationException -import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils} -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.Driver -import org.apache.hadoop.hive.ql.exec.Utilities -import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory} -import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.hive.shims.ShimLoader -import org.apache.thrift.transport.TSocket - -import org.apache.spark.sql.Logging - -private[hive] object SparkSQLCLIDriver { - private var prompt = "spark-sql" - private var continuedPrompt = "".padTo(prompt.length, ' ') - private var transport:TSocket = _ - - installSignalHandler() - - /** - * Install an interrupt callback to cancel all Spark jobs. In Hive's CliDriver#processLine(), - * a signal handler will invoke this registered callback if a Ctrl+C signal is detected while - * a command is being processed by the current thread. - */ - def installSignalHandler() { - HiveInterruptUtils.add(new HiveInterruptCallback { - override def interrupt() { - // Handle remote execution mode - if (SparkSQLEnv.sparkContext != null) { - SparkSQLEnv.sparkContext.cancelAllJobs() - } else { - if (transport != null) { - // Force closing of TCP connection upon session termination - transport.getSocket.close() - } - } - } - }) - } - - def main(args: Array[String]) { - val oproc = new OptionsProcessor() - if (!oproc.process_stage1(args)) { - System.exit(1) - } - - // NOTE: It is critical to do this here so that log4j is reinitialized - // before any of the other core hive classes are loaded - var logInitFailed = false - var logInitDetailMessage: String = null - try { - logInitDetailMessage = LogUtils.initHiveLog4j() - } catch { - case e: LogInitializationException => - logInitFailed = true - logInitDetailMessage = e.getMessage - } - - val sessionState = new CliSessionState(new HiveConf(classOf[SessionState])) - - sessionState.in = System.in - try { - sessionState.out = new PrintStream(System.out, true, "UTF-8") - sessionState.info = new PrintStream(System.err, true, "UTF-8") - sessionState.err = new PrintStream(System.err, true, "UTF-8") - } catch { - case e: UnsupportedEncodingException => System.exit(3) - } - - if (!oproc.process_stage2(sessionState)) { - System.exit(2) - } - - if (!sessionState.getIsSilent) { - if (logInitFailed) System.err.println(logInitDetailMessage) - else SessionState.getConsole.printInfo(logInitDetailMessage) - } - - // Set all properties specified via command line. - val conf: HiveConf = sessionState.getConf - sessionState.cmdProperties.entrySet().foreach { item: java.util.Map.Entry[Object, Object] => - conf.set(item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) - sessionState.getOverriddenConfigurations.put( - item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) - } - - SessionState.start(sessionState) - - // Clean up after we exit - Runtime.getRuntime.addShutdownHook( - new Thread() { - override def run() { - SparkSQLEnv.stop() - } - } - ) - - // "-h" option has been passed, so connect to Hive thrift server. - if (sessionState.getHost != null) { - sessionState.connect() - if (sessionState.isRemoteMode) { - prompt = s"[${sessionState.getHost}:${sessionState.getPort}]" + prompt - continuedPrompt = "".padTo(prompt.length, ' ') - } - } - - if (!sessionState.isRemoteMode && !ShimLoader.getHadoopShims.usesJobShell()) { - // Hadoop-20 and above - we need to augment classpath using hiveconf - // components. - // See also: code in ExecDriver.java - var loader = conf.getClassLoader - val auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEAUXJARS) - if (StringUtils.isNotBlank(auxJars)) { - loader = Utilities.addToClassPath(loader, StringUtils.split(auxJars, ",")) - } - conf.setClassLoader(loader) - Thread.currentThread().setContextClassLoader(loader) - } - - val cli = new SparkSQLCLIDriver - cli.setHiveVariables(oproc.getHiveVariables) - - // TODO work around for set the log output to console, because the HiveContext - // will set the output into an invalid buffer. - sessionState.in = System.in - try { - sessionState.out = new PrintStream(System.out, true, "UTF-8") - sessionState.info = new PrintStream(System.err, true, "UTF-8") - sessionState.err = new PrintStream(System.err, true, "UTF-8") - } catch { - case e: UnsupportedEncodingException => System.exit(3) - } - - // Execute -i init files (always in silent mode) - cli.processInitFiles(sessionState) - - if (sessionState.execString != null) { - System.exit(cli.processLine(sessionState.execString)) - } - - try { - if (sessionState.fileName != null) { - System.exit(cli.processFile(sessionState.fileName)) - } - } catch { - case e: FileNotFoundException => - System.err.println(s"Could not open input file for reading. (${e.getMessage})") - System.exit(3) - } - - val reader = new ConsoleReader() - reader.setBellEnabled(false) - // reader.setDebug(new PrintWriter(new FileWriter("writer.debug", true))) - CliDriver.getCommandCompletor.foreach((e) => reader.addCompletor(e)) - - val historyDirectory = System.getProperty("user.home") - - try { - if (new File(historyDirectory).exists()) { - val historyFile = historyDirectory + File.separator + ".hivehistory" - reader.setHistory(new History(new File(historyFile))) - } else { - System.err.println("WARNING: Directory for Hive history file: " + historyDirectory + - " does not exist. History will not be available during this session.") - } - } catch { - case e: Exception => - System.err.println("WARNING: Encountered an error while trying to initialize Hive's " + - "history file. History will not be available during this session.") - System.err.println(e.getMessage) - } - - val clientTransportTSocketField = classOf[CliSessionState].getDeclaredField("transport") - clientTransportTSocketField.setAccessible(true) - - transport = clientTransportTSocketField.get(sessionState).asInstanceOf[TSocket] - - var ret = 0 - var prefix = "" - val currentDB = ReflectionUtils.invokeStatic(classOf[CliDriver], "getFormattedDb", - classOf[HiveConf] -> conf, classOf[CliSessionState] -> sessionState) - - def promptWithCurrentDB = s"$prompt$currentDB" - def continuedPromptWithDBSpaces = continuedPrompt + ReflectionUtils.invokeStatic( - classOf[CliDriver], "spacesForString", classOf[String] -> currentDB) - - var currentPrompt = promptWithCurrentDB - var line = reader.readLine(currentPrompt + "> ") - - while (line != null) { - if (prefix.nonEmpty) { - prefix += '\n' - } - - if (line.trim().endsWith(";") && !line.trim().endsWith("\\;")) { - line = prefix + line - ret = cli.processLine(line, true) - prefix = "" - currentPrompt = promptWithCurrentDB - } else { - prefix = prefix + line - currentPrompt = continuedPromptWithDBSpaces - } - - line = reader.readLine(currentPrompt + "> ") - } - - sessionState.close() - - System.exit(ret) - } -} - -private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { - private val sessionState = SessionState.get().asInstanceOf[CliSessionState] - - private val LOG = LogFactory.getLog("CliDriver") - - private val console = new SessionState.LogHelper(LOG) - - private val conf: Configuration = - if (sessionState != null) sessionState.getConf else new Configuration() - - // Force initializing SparkSQLEnv. This is put here but not object SparkSQLCliDriver - // because the Hive unit tests do not go through the main() code path. - if (!sessionState.isRemoteMode) { - SparkSQLEnv.init() - } - - override def processCmd(cmd: String): Int = { - val cmd_trimmed: String = cmd.trim() - val tokens: Array[String] = cmd_trimmed.split("\\s+") - val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() - if (cmd_trimmed.toLowerCase.equals("quit") || - cmd_trimmed.toLowerCase.equals("exit") || - tokens(0).equalsIgnoreCase("source") || - cmd_trimmed.startsWith("!") || - tokens(0).toLowerCase.equals("list") || - sessionState.isRemoteMode) { - val start = System.currentTimeMillis() - super.processCmd(cmd) - val end = System.currentTimeMillis() - val timeTaken: Double = (end - start) / 1000.0 - console.printInfo(s"Time taken: $timeTaken seconds") - 0 - } else { - var ret = 0 - val hconf = conf.asInstanceOf[HiveConf] - val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hconf) - - if (proc != null) { - if (proc.isInstanceOf[Driver]) { - val driver = new SparkSQLDriver - - driver.init() - val out = sessionState.out - val start:Long = System.currentTimeMillis() - if (sessionState.getIsVerbose) { - out.println(cmd) - } - - ret = driver.run(cmd).getResponseCode - if (ret != 0) { - driver.close() - return ret - } - - val res = new JArrayList[String]() - - if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER)) { - // Print the column names. - Option(driver.getSchema.getFieldSchemas).map { fields => - out.println(fields.map(_.getName).mkString("\t")) - } - } - - try { - while (!out.checkError() && driver.getResults(res)) { - res.foreach(out.println) - res.clear() - } - } catch { - case e:IOException => - console.printError( - s"""Failed with exception ${e.getClass.getName}: ${e.getMessage} - |${org.apache.hadoop.util.StringUtils.stringifyException(e)} - """.stripMargin) - ret = 1 - } - - val cret = driver.close() - if (ret == 0) { - ret = cret - } - - val end = System.currentTimeMillis() - if (end > start) { - val timeTaken:Double = (end - start) / 1000.0 - console.printInfo(s"Time taken: $timeTaken seconds", null) - } - - // Destroy the driver to release all the locks. - driver.destroy() - } else { - if (sessionState.getIsVerbose) { - sessionState.out.println(tokens(0) + " " + cmd_1) - } - ret = proc.run(cmd_1).getResponseCode - } - } - ret - } - } -} - diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala deleted file mode 100644 index 42cbf363b274f..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ /dev/null @@ -1,74 +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.sql.hive.thriftserver - -import scala.collection.JavaConversions._ - -import java.io.IOException -import java.util.{List => JList} -import javax.security.auth.login.LoginException - -import org.apache.commons.logging.Log -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.shims.ShimLoader -import org.apache.hive.service.Service.STATE -import org.apache.hive.service.auth.HiveAuthFactory -import org.apache.hive.service.cli.CLIService -import org.apache.hive.service.{AbstractService, Service, ServiceException} - -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ - -private[hive] class SparkSQLCLIService(hiveContext: HiveContext) - extends CLIService - with ReflectedCompositeService { - - override def init(hiveConf: HiveConf) { - setSuperField(this, "hiveConf", hiveConf) - - val sparkSqlSessionManager = new SparkSQLSessionManager(hiveContext) - setSuperField(this, "sessionManager", sparkSqlSessionManager) - addService(sparkSqlSessionManager) - - try { - HiveAuthFactory.loginFromKeytab(hiveConf) - val serverUserName = ShimLoader.getHadoopShims - .getShortUserName(ShimLoader.getHadoopShims.getUGIForConf(hiveConf)) - setSuperField(this, "serverUserName", serverUserName) - } catch { - case e @ (_: IOException | _: LoginException) => - throw new ServiceException("Unable to login to kerberos with given principal/keytab", e) - } - - initCompositeService(hiveConf) - } -} - -private[thriftserver] trait ReflectedCompositeService { this: AbstractService => - def initCompositeService(hiveConf: HiveConf) { - // Emulating `CompositeService.init(hiveConf)` - val serviceList = getAncestorField[JList[Service]](this, 2, "serviceList") - serviceList.foreach(_.init(hiveConf)) - - // Emulating `AbstractService.init(hiveConf)` - invoke(classOf[AbstractService], this, "ensureCurrentState", classOf[STATE] -> STATE.NOTINITED) - setAncestorField(this, 3, "hiveConf", hiveConf) - invoke(classOf[AbstractService], this, "changeState", classOf[STATE] -> STATE.INITED) - getAncestorField[Log](this, 3, "LOG").info(s"Service: $getName is inited.") - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala deleted file mode 100644 index 5202aa9903e03..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ /dev/null @@ -1,93 +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.sql.hive.thriftserver - -import scala.collection.JavaConversions._ - -import java.util.{ArrayList => JArrayList} - -import org.apache.commons.lang.exception.ExceptionUtils -import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} -import org.apache.hadoop.hive.ql.Driver -import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse - -import org.apache.spark.sql.Logging -import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} - -private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveContext) - extends Driver with Logging { - - private var tableSchema: Schema = _ - private var hiveResponse: Seq[String] = _ - - override def init(): Unit = { - } - - private def getResultSetSchema(query: context.QueryExecution): Schema = { - val analyzed = query.analyzed - logger.debug(s"Result Schema: ${analyzed.output}") - if (analyzed.output.size == 0) { - new Schema(new FieldSchema("Response code", "string", "") :: Nil, null) - } else { - val fieldSchemas = analyzed.output.map { attr => - new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") - } - - new Schema(fieldSchemas, null) - } - } - - override def run(command: String): CommandProcessorResponse = { - val execution = context.executePlan(context.hql(command).logicalPlan) - - // TODO unify the error code - try { - hiveResponse = execution.stringResult() - tableSchema = getResultSetSchema(execution) - new CommandProcessorResponse(0) - } catch { - case cause: Throwable => - logger.error(s"Failed in [$command]", cause) - new CommandProcessorResponse(-3, ExceptionUtils.getFullStackTrace(cause), null) - } - } - - override def close(): Int = { - hiveResponse = null - tableSchema = null - 0 - } - - override def getSchema: Schema = tableSchema - - override def getResults(res: JArrayList[String]): Boolean = { - if (hiveResponse == null) { - false - } else { - res.addAll(hiveResponse) - hiveResponse = null - true - } - } - - override def destroy() { - super.destroy() - hiveResponse = null - tableSchema = null - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala deleted file mode 100644 index 451c3bd7b9352..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ /dev/null @@ -1,58 +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.sql.hive.thriftserver - -import org.apache.hadoop.hive.ql.session.SessionState - -import org.apache.spark.scheduler.{SplitInfo, StatsReportListener} -import org.apache.spark.sql.Logging -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.{SparkConf, SparkContext} - -/** A singleton object for the master program. The slaves should not access this. */ -private[hive] object SparkSQLEnv extends Logging { - logger.debug("Initializing SparkSQLEnv") - - var hiveContext: HiveContext = _ - var sparkContext: SparkContext = _ - - def init() { - if (hiveContext == null) { - sparkContext = new SparkContext(new SparkConf() - .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}")) - - sparkContext.addSparkListener(new StatsReportListener()) - - hiveContext = new HiveContext(sparkContext) { - @transient override lazy val sessionState = SessionState.get() - @transient override lazy val hiveconf = sessionState.getConf - } - } - } - - /** Cleans up and shuts down the Spark SQL environments. */ - def stop() { - logger.debug("Shutting down Spark SQL Environment") - // Stop the SparkContext - if (SparkSQLEnv.sparkContext != null) { - sparkContext.stop() - sparkContext = null - hiveContext = null - } - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala deleted file mode 100644 index 6b3275b4eaf04..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ /dev/null @@ -1,49 +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.sql.hive.thriftserver - -import java.util.concurrent.Executors - -import org.apache.commons.logging.Log -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hive.service.cli.session.SessionManager - -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ -import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager - -private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) - extends SessionManager - with ReflectedCompositeService { - - override def init(hiveConf: HiveConf) { - setSuperField(this, "hiveConf", hiveConf) - - val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS) - setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize)) - getAncestorField[Log](this, 3, "LOG").info( - s"HiveServer2: Async execution pool size $backgroundPoolSize") - - val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext) - setSuperField(this, "operationManager", sparkSqlOperationManager) - addService(sparkSqlOperationManager) - - initCompositeService(hiveConf) - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala deleted file mode 100644 index a4e1f3e762e89..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ /dev/null @@ -1,151 +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.sql.hive.thriftserver.server - -import scala.collection.JavaConversions._ -import scala.collection.mutable.ArrayBuffer -import scala.math.{random, round} - -import java.sql.Timestamp -import java.util.{Map => JMap} - -import org.apache.hadoop.hive.common.`type`.HiveDecimal -import org.apache.hadoop.hive.metastore.api.FieldSchema -import org.apache.hive.service.cli._ -import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} -import org.apache.hive.service.cli.session.HiveSession - -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils -import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} -import org.apache.spark.sql.{Logging, SchemaRDD, Row => SparkRow} - -/** - * Executes queries using Spark SQL, and maintains a list of handles to active queries. - */ -class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManager with Logging { - val handleToOperation = ReflectionUtils - .getSuperField[JMap[OperationHandle, Operation]](this, "handleToOperation") - - override def newExecuteStatementOperation( - parentSession: HiveSession, - statement: String, - confOverlay: JMap[String, String], - async: Boolean): ExecuteStatementOperation = synchronized { - - val operation = new ExecuteStatementOperation(parentSession, statement, confOverlay) { - private var result: SchemaRDD = _ - private var iter: Iterator[SparkRow] = _ - private var dataTypes: Array[DataType] = _ - - def close(): Unit = { - // RDDs will be cleaned automatically upon garbage collection. - logger.debug("CLOSING") - } - - def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { - if (!iter.hasNext) { - new RowSet() - } else { - val maxRows = maxRowsL.toInt // Do you really want a row batch larger than Int Max? No. - var curRow = 0 - var rowSet = new ArrayBuffer[Row](maxRows) - - while (curRow < maxRows && iter.hasNext) { - val sparkRow = iter.next() - val row = new Row() - var curCol = 0 - - while (curCol < sparkRow.length) { - dataTypes(curCol) match { - case StringType => - row.addString(sparkRow(curCol).asInstanceOf[String]) - case IntegerType => - row.addColumnValue(ColumnValue.intValue(sparkRow.getInt(curCol))) - case BooleanType => - row.addColumnValue(ColumnValue.booleanValue(sparkRow.getBoolean(curCol))) - case DoubleType => - row.addColumnValue(ColumnValue.doubleValue(sparkRow.getDouble(curCol))) - case FloatType => - row.addColumnValue(ColumnValue.floatValue(sparkRow.getFloat(curCol))) - case DecimalType => - val hiveDecimal = sparkRow.get(curCol).asInstanceOf[BigDecimal].bigDecimal - row.addColumnValue(ColumnValue.stringValue(new HiveDecimal(hiveDecimal))) - case LongType => - row.addColumnValue(ColumnValue.longValue(sparkRow.getLong(curCol))) - case ByteType => - row.addColumnValue(ColumnValue.byteValue(sparkRow.getByte(curCol))) - case ShortType => - row.addColumnValue(ColumnValue.intValue(sparkRow.getShort(curCol))) - case TimestampType => - row.addColumnValue( - ColumnValue.timestampValue(sparkRow.get(curCol).asInstanceOf[Timestamp])) - case BinaryType | _: ArrayType | _: StructType | _: MapType => - val hiveString = result - .queryExecution - .asInstanceOf[HiveContext#QueryExecution] - .toHiveString((sparkRow.get(curCol), dataTypes(curCol))) - row.addColumnValue(ColumnValue.stringValue(hiveString)) - } - curCol += 1 - } - rowSet += row - curRow += 1 - } - new RowSet(rowSet, 0) - } - } - - def getResultSetSchema: TableSchema = { - logger.warn(s"Result Schema: ${result.queryExecution.analyzed.output}") - if (result.queryExecution.analyzed.output.size == 0) { - new TableSchema(new FieldSchema("Result", "string", "") :: Nil) - } else { - val schema = result.queryExecution.analyzed.output.map { attr => - new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") - } - new TableSchema(schema) - } - } - - def run(): Unit = { - logger.info(s"Running query '$statement'") - setState(OperationState.RUNNING) - try { - result = hiveContext.hql(statement) - logger.debug(result.queryExecution.toString()) - val groupId = round(random * 1000000).toString - hiveContext.sparkContext.setJobGroup(groupId, statement) - iter = result.queryExecution.toRdd.toLocalIterator - dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray - setHasResultSet(true) - } catch { - // Actually do need to catch Throwable as some failures don't inherit from Exception and - // HiveServer will silently swallow them. - case e: Throwable => - logger.error("Error executing query:",e) - throw new HiveSQLException(e.toString) - } - setState(OperationState.FINISHED) - } - } - - handleToOperation.put(operation.getHandle, operation) - operation - } -} diff --git a/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt b/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt deleted file mode 100644 index 850f8014b6f05..0000000000000 --- a/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt +++ /dev/null @@ -1,5 +0,0 @@ -238val_238 -86val_86 -311val_311 -27val_27 -165val_165 diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala deleted file mode 100644 index b90670a796b81..0000000000000 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ /dev/null @@ -1,59 +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.sql.hive.thriftserver - -import java.io.{BufferedReader, InputStreamReader, PrintWriter} - -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import org.apache.spark.sql.hive.test.TestHive - -class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { - val WAREHOUSE_PATH = TestUtils.getWarehousePath("cli") - val METASTORE_PATH = TestUtils.getMetastorePath("cli") - - override def beforeAll() { - val pb = new ProcessBuilder( - "../../bin/spark-sql", - "--master", - "local", - "--hiveconf", - s"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$METASTORE_PATH;create=true", - "--hiveconf", - "hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) - - process = pb.start() - outputWriter = new PrintWriter(process.getOutputStream, true) - inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) - errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) - waitForOutput(inputReader, "spark-sql>") - } - - override def afterAll() { - process.destroy() - process.waitFor() - } - - test("simple commands") { - val dataFilePath = getDataFile("data/files/small_kv.txt") - executeQuery("create table hive_test1(key int, val string);") - executeQuery("load data local inpath '" + dataFilePath+ "' overwrite into table hive_test1;") - executeQuery("cache table hive_test1", "Time taken") - } -} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala deleted file mode 100644 index 59f4952b78bc6..0000000000000 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ /dev/null @@ -1,125 +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.sql.hive.thriftserver - -import scala.collection.JavaConversions._ -import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent._ - -import java.io.{BufferedReader, InputStreamReader} -import java.sql.{Connection, DriverManager, Statement} - -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import org.apache.spark.sql.Logging -import org.apache.spark.sql.catalyst.util.getTempFilePath - -/** - * Test for the HiveThriftServer2 using JDBC. - */ -class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUtils with Logging { - - val WAREHOUSE_PATH = getTempFilePath("warehouse") - val METASTORE_PATH = getTempFilePath("metastore") - - val DRIVER_NAME = "org.apache.hive.jdbc.HiveDriver" - val TABLE = "test" - // use a different port, than the hive standard 10000, - // for tests to avoid issues with the port being taken on some machines - val PORT = "10000" - - // If verbose is true, the test program will print all outputs coming from the Hive Thrift server. - val VERBOSE = Option(System.getenv("SPARK_SQL_TEST_VERBOSE")).getOrElse("false").toBoolean - - Class.forName(DRIVER_NAME) - - override def beforeAll() { launchServer() } - - override def afterAll() { stopServer() } - - private def launchServer(args: Seq[String] = Seq.empty) { - // Forking a new process to start the Hive Thrift server. The reason to do this is it is - // hard to clean up Hive resources entirely, so we just start a new process and kill - // that process for cleanup. - val defaultArgs = Seq( - "../../sbin/start-thriftserver.sh", - "--master local", - "--hiveconf", - "hive.root.logger=INFO,console", - "--hiveconf", - s"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$METASTORE_PATH;create=true", - "--hiveconf", - s"hive.metastore.warehouse.dir=$WAREHOUSE_PATH") - val pb = new ProcessBuilder(defaultArgs ++ args) - process = pb.start() - inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) - errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) - waitForOutput(inputReader, "ThriftBinaryCLIService listening on") - - // Spawn a thread to read the output from the forked process. - // Note that this is necessary since in some configurations, log4j could be blocked - // if its output to stderr are not read, and eventually blocking the entire test suite. - future { - while (true) { - val stdout = readFrom(inputReader) - val stderr = readFrom(errorReader) - if (VERBOSE && stdout.length > 0) { - println(stdout) - } - if (VERBOSE && stderr.length > 0) { - println(stderr) - } - Thread.sleep(50) - } - } - } - - private def stopServer() { - process.destroy() - process.waitFor() - } - - test("test query execution against a Hive Thrift server") { - Thread.sleep(5 * 1000) - val dataFilePath = getDataFile("data/files/small_kv.txt") - val stmt = createStatement() - stmt.execute("DROP TABLE IF EXISTS test") - stmt.execute("DROP TABLE IF EXISTS test_cached") - stmt.execute("CREATE TABLE test(key int, val string)") - stmt.execute(s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test") - stmt.execute("CREATE TABLE test_cached as select * from test limit 4") - stmt.execute("CACHE TABLE test_cached") - - var rs = stmt.executeQuery("select count(*) from test") - rs.next() - assert(rs.getInt(1) === 5) - - rs = stmt.executeQuery("select count(*) from test_cached") - rs.next() - assert(rs.getInt(1) === 4) - - stmt.close() - } - - def getConnection: Connection = { - val connectURI = s"jdbc:hive2://localhost:$PORT/" - DriverManager.getConnection(connectURI, System.getProperty("user.name"), "") - } - - def createStatement(): Statement = getConnection.createStatement() -} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala deleted file mode 100644 index bb2242618fbef..0000000000000 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala +++ /dev/null @@ -1,108 +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.sql.hive.thriftserver - -import java.io.{BufferedReader, PrintWriter} -import java.text.SimpleDateFormat -import java.util.Date - -import org.apache.hadoop.hive.common.LogUtils -import org.apache.hadoop.hive.common.LogUtils.LogInitializationException - -object TestUtils { - val timestamp = new SimpleDateFormat("yyyyMMdd-HHmmss") - - def getWarehousePath(prefix: String): String = { - System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-warehouse-" + - timestamp.format(new Date) - } - - def getMetastorePath(prefix: String): String = { - System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-metastore-" + - timestamp.format(new Date) - } - - // Dummy function for initialize the log4j properties. - def init() { } - - // initialize log4j - try { - LogUtils.initHiveLog4j() - } catch { - case e: LogInitializationException => // Ignore the error. - } -} - -trait TestUtils { - var process : Process = null - var outputWriter : PrintWriter = null - var inputReader : BufferedReader = null - var errorReader : BufferedReader = null - - def executeQuery( - cmd: String, outputMessage: String = "OK", timeout: Long = 15000): String = { - println("Executing: " + cmd + ", expecting output: " + outputMessage) - outputWriter.write(cmd + "\n") - outputWriter.flush() - waitForQuery(timeout, outputMessage) - } - - protected def waitForQuery(timeout: Long, message: String): String = { - if (waitForOutput(errorReader, message, timeout)) { - Thread.sleep(500) - readOutput() - } else { - assert(false, "Didn't find \"" + message + "\" in the output:\n" + readOutput()) - null - } - } - - // Wait for the specified str to appear in the output. - protected def waitForOutput( - reader: BufferedReader, str: String, timeout: Long = 10000): Boolean = { - val startTime = System.currentTimeMillis - var out = "" - while (!out.contains(str) && System.currentTimeMillis < (startTime + timeout)) { - out += readFrom(reader) - } - out.contains(str) - } - - // Read stdout output and filter out garbage collection messages. - protected def readOutput(): String = { - val output = readFrom(inputReader) - // Remove GC Messages - val filteredOutput = output.lines.filterNot(x => x.contains("[GC") || x.contains("[Full GC")) - .mkString("\n") - filteredOutput - } - - protected def readFrom(reader: BufferedReader): String = { - var out = "" - var c = 0 - while (reader.ready) { - c = reader.read() - out += c.asInstanceOf[Char] - } - out - } - - protected def getDataFile(name: String) = { - Thread.currentThread().getContextClassLoader.getResource(name) - } -} diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 93d00f7c37c9b..1699ffe06ce15 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -32,7 +32,7 @@ Spark Project Hive http://spark.apache.org/ - hive + hive 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 84d43eaeea51d..201c85f3d501e 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 @@ -255,7 +255,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, ShortType, DecimalType, TimestampType, BinaryType) - protected[sql] def toHiveString(a: (Any, DataType)): String = a match { + protected def toHiveString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => struct.zip(fields).map { case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 4395874526d51..e6ab68b563f8d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -925,7 +925,8 @@ private[hive] object HiveQl { case Token("-", left :: right:: Nil) => Subtract(nodeToExpr(left), nodeToExpr(right)) case Token("*", left :: right:: Nil) => Multiply(nodeToExpr(left), nodeToExpr(right)) case Token("/", left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right)) - case Token(DIV(), left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right)) + case Token(DIV(), left :: right:: Nil) => + Cast(Divide(nodeToExpr(left), nodeToExpr(right)), LongType) case Token("%", left :: right:: Nil) => Remainder(nodeToExpr(left), nodeToExpr(right)) /* Comparisons */ diff --git a/sql/hive/src/test/resources/golden/div-0-3760f9b354ddacd7c7b01b28791d4585 b/sql/hive/src/test/resources/golden/div-0-3760f9b354ddacd7c7b01b28791d4585 new file mode 100644 index 0000000000000..17ba0bea723c6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/div-0-3760f9b354ddacd7c7b01b28791d4585 @@ -0,0 +1 @@ +0 0 0 1 2 diff --git a/sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 b/sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 new file mode 100644 index 0000000000000..7b7a9175114ce --- /dev/null +++ b/sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 @@ -0,0 +1 @@ +2.0 0.5 0.3333333333333333 0.002 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 08ef4d9b6bb93..b4dbf2b115799 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 @@ -350,12 +350,6 @@ abstract class HiveComparisonTest val resultComparison = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n") - println("hive output") - hive.foreach(println) - - println("catalyst printout") - catalyst.foreach(println) - if (recomputeCache) { logger.warn(s"Clearing cache files for failed test $testCaseName") hiveCacheFiles.foreach(_.delete()) 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 8489f2a34e63c..a8623b64c656f 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 @@ -52,7 +52,10 @@ class HiveQuerySuite extends HiveComparisonTest { "SELECT * FROM src WHERE key Between 1 and 2") createQueryTest("div", - "SELECT 1 DIV 2, 1 div 2, 1 dIv 2 FROM src LIMIT 1") + "SELECT 1 DIV 2, 1 div 2, 1 dIv 2, 100 DIV 51, 100 DIV 49 FROM src LIMIT 1") + + createQueryTest("division", + "SELECT 2 / 1, 1 / 2, 1 / 3, 1 / COUNT(*) FROM src LIMIT 1") test("Query expressed in SQL") { assert(sql("SELECT 1").collect() === Array(Seq(1))) @@ -416,10 +419,10 @@ class HiveQuerySuite extends HiveComparisonTest { hql(s"set $testKey=$testVal") assert(get(testKey, testVal + "_") == testVal) - hql("set some.property=20") - assert(get("some.property", "0") == "20") - hql("set some.property = 40") - assert(get("some.property", "0") == "40") + hql("set mapred.reduce.tasks=20") + assert(get("mapred.reduce.tasks", "0") == "20") + hql("set mapred.reduce.tasks = 40") + assert(get("mapred.reduce.tasks", "0") == "40") hql(s"set $testKey=$testVal") assert(get(testKey, "0") == testVal) @@ -433,61 +436,63 @@ class HiveQuerySuite extends HiveComparisonTest { val testKey = "spark.sql.key.usedfortestonly" val testVal = "test.val.0" val nonexistentKey = "nonexistent" + def collectResults(rdd: SchemaRDD): Set[(String, String)] = + rdd.collect().map { case Row(key: String, value: String) => key -> value }.toSet clear() // "set" itself returns all config variables currently specified in SQLConf. assert(hql("SET").collect().size == 0) - assertResult(Array(s"$testKey=$testVal")) { - hql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(s"$testKey=$testVal")) { - hql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql("SET")) } hql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { - hql(s"SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { + collectResults(hql("SET")) } // "set key" - assertResult(Array(s"$testKey=$testVal")) { - hql(s"SET $testKey").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql(s"SET $testKey")) } - assertResult(Array(s"$nonexistentKey=")) { - hql(s"SET $nonexistentKey").collect().map(_.getString(0)) + assertResult(Set(nonexistentKey -> "")) { + collectResults(hql(s"SET $nonexistentKey")) } // Assert that sql() should have the same effects as hql() by repeating the above using sql(). clear() assert(sql("SET").collect().size == 0) - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(s"$testKey=$testVal")) { - sql("SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql("SET")) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { - sql("SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { + collectResults(sql("SET")) } - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql(s"SET $testKey")) } - assertResult(Array(s"$nonexistentKey=")) { - sql(s"SET $nonexistentKey").collect().map(_.getString(0)) + assertResult(Set(nonexistentKey -> "")) { + collectResults(sql(s"SET $nonexistentKey")) } clear() diff --git a/streaming/pom.xml b/streaming/pom.xml index b99f306b8f2cc..f60697ce745b7 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming_2.10 - streaming + streaming jar Spark Project Streaming diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index ce8316bb14891..d934b9cbfc3e8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -110,8 +110,7 @@ private[streaming] class ReceiverSupervisorImpl( ) { val blockId = optionalBlockId.getOrElse(nextBlockId) val time = System.currentTimeMillis - blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], - storageLevel, tellMaster = true) + blockManager.putArray(blockId, arrayBuffer.toArray[Any], storageLevel, tellMaster = true) logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") reportPushedBlock(blockId, arrayBuffer.size, optionalMetadata) } @@ -124,7 +123,7 @@ private[streaming] class ReceiverSupervisorImpl( ) { val blockId = optionalBlockId.getOrElse(nextBlockId) val time = System.currentTimeMillis - blockManager.put(blockId, iterator, storageLevel, tellMaster = true) + blockManager.putIterator(blockId, iterator, storageLevel, tellMaster = true) logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") reportPushedBlock(blockId, -1, optionalMetadata) } diff --git a/tools/pom.xml b/tools/pom.xml index 97abb6b2b63e0..c0ee8faa7a615 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -27,7 +27,7 @@ org.apache.spark spark-tools_2.10 - tools + tools jar Spark Project Tools diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index 51744ece0412d..5b13a1f002d6e 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -24,7 +24,7 @@ ../pom.xml - yarn-alpha + yarn-alpha org.apache.spark diff --git a/yarn/pom.xml b/yarn/pom.xml index 3faaf053634d6..efb473aa1b261 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -29,7 +29,7 @@ pom Spark Project YARN Parent POM - yarn + yarn diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index b6c8456d06684..ceaf9f9d71001 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -24,7 +24,7 @@ ../pom.xml - yarn-stable + yarn-stable org.apache.spark