From 0ac71d1284cd4f011d5763181cba9ecb49337b66 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 20 Jun 2014 15:38:02 -0700 Subject: [PATCH 001/124] [SPARK-2225] Turn HAVING without GROUP BY into WHERE. @willb Author: Reynold Xin Closes #1161 from rxin/having-filter and squashes the following commits: fa8359a [Reynold Xin] [SPARK-2225] Turn HAVING without GROUP BY into WHERE. --- .../org/apache/spark/sql/hive/HiveQl.scala | 15 ++++----------- .../sql/hive/execution/HiveQuerySuite.scala | 19 +++++++------------ 2 files changed, 11 insertions(+), 23 deletions(-) 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 c69e3dba6b467..b073dc3895f05 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 @@ -563,7 +563,6 @@ private[hive] object HiveQl { withWhere) }.getOrElse(withWhere) - // The projection of the query can either be a normal projection, an aggregation // (if there is a group by) or a script transformation. val withProject = transformation.getOrElse { @@ -581,16 +580,10 @@ private[hive] object HiveQl { val withDistinct = if (selectDistinctClause.isDefined) Distinct(withProject) else withProject - val withHaving = havingClause.map { h => - - if (groupByClause == None) { - throw new SemanticException("HAVING specified without GROUP BY") - } - - val havingExpr = h.getChildren.toSeq match { - case Seq(hexpr) => nodeToExpr(hexpr) - } - + val withHaving = havingClause.map { h => + val havingExpr = h.getChildren.toSeq match { case Seq(hexpr) => nodeToExpr(hexpr) } + // Note that we added a cast to boolean. If the expression itself is already boolean, + // the optimizer will get rid of the unnecessary cast. Filter(Cast(havingExpr, BooleanType), withDistinct) }.getOrElse(withDistinct) 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 80185098bf24f..d855310253bf3 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 @@ -227,27 +227,22 @@ class HiveQuerySuite extends HiveComparisonTest { test("SPARK-2180: HAVING support in GROUP BY clauses (positive)") { val fixture = List(("foo", 2), ("bar", 1), ("foo", 4), ("bar", 3)) .zipWithIndex.map {case Pair(Pair(value, attr), key) => HavingRow(key, value, attr)} - TestHive.sparkContext.parallelize(fixture).registerAsTable("having_test") - val results = hql("SELECT value, max(attr) AS attr FROM having_test GROUP BY value HAVING attr > 3") .collect() .map(x => Pair(x.getString(0), x.getInt(1))) - + assert(results === Array(Pair("foo", 4))) - TestHive.reset() } - - test("SPARK-2180: HAVING without GROUP BY raises exception") { - intercept[Exception] { - hql("SELECT value, attr FROM having_test HAVING attr > 3") - } - } - test("SPARK-2180: HAVING with non-boolean clause raises no exceptions") { - val results = hql("select key, count(*) c from src group by key having c").collect() + test("SPARK-2180: HAVING with non-boolean clause raises no exceptions") { + hql("select key, count(*) c from src group by key having c").collect() + } + + test("SPARK-2225: turn HAVING without GROUP BY into a simple filter") { + assert(hql("select key from src having key > 490").collect().size < 100) } test("Query Hive native command execution result") { From 01125a1162b7bf42ae907cb6616616cc4ffb6fa9 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 20 Jun 2014 17:14:33 -0700 Subject: [PATCH 002/124] Clean up CacheManager et al. **UPDATE** I have removed the special handling for `StorageLevel.MEMORY_*_SER` for now, because it introduces a potential performance regression. With the latest changes, this PR should include mainly style (code readability) fixes. The only functionality change is the update in `MemoryStore#putBytes` to actually return updated blocks, though this is a minor bug fix. Now this is mainly a precursor to another PR (once again). --------- *Old comment* The deserialized version of a partition may occupy much more space than the serialized version. Therefore, if a partition is to be cached with `StorageLevel.MEMORY_*_SER`, we don't need to fully unroll it into an `ArrayBuffer`, but instead we can unroll it into a potentially much smaller `ByteBuffer`. This may save us from OOMs in this case. Author: Andrew Or Closes #1083 from andrewor14/unroll-them-partitions and squashes the following commits: 7048aa0 [Andrew Or] Merge branch 'master' of github.com:apache/spark into unroll-them-partitions 3d9a366 [Andrew Or] Minor change for readability d12b95f [Andrew Or] Remove unused imports (minor) a4c387b [Andrew Or] Merge branch 'master' of github.com:apache/spark into unroll-them-partitions cf5f565 [Andrew Or] Remove special handling for MEM_*_SER 0091ec0 [Andrew Or] Address review feedback 44ef282 [Andrew Or] Actually return updated blocks in putBytes 2941c89 [Andrew Or] Clean up BlockStore (minor) a8f181d [Andrew Or] Add special handling for StorageLevel.MEMORY_*_SER --- .../scala/org/apache/spark/CacheManager.scala | 163 ++++++++++-------- .../apache/spark/scheduler/ResultTask.scala | 3 +- .../spark/scheduler/ShuffleMapTask.scala | 5 +- .../org/apache/spark/storage/BlockStore.scala | 22 ++- .../apache/spark/storage/MemoryStore.scala | 8 +- 5 files changed, 113 insertions(+), 88 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 315ed91f81df3..3f667a4a0f9c5 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -20,25 +20,25 @@ package org.apache.spark import scala.collection.mutable.{ArrayBuffer, HashSet} import org.apache.spark.rdd.RDD -import org.apache.spark.storage.{BlockId, BlockManager, BlockStatus, RDDBlockId, StorageLevel} +import org.apache.spark.storage._ /** - * Spark class responsible for passing RDDs split contents to the BlockManager and making + * Spark class responsible for passing RDDs partition contents to the BlockManager and making * sure a node doesn't load two copies of an RDD at once. */ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { - /** Keys of RDD splits that are being computed/loaded. */ + /** Keys of RDD partitions that are being computed/loaded. */ private val loading = new HashSet[RDDBlockId]() - /** Gets or computes an RDD split. Used by RDD.iterator() when an RDD is cached. */ + /** Gets or computes an RDD partition. Used by RDD.iterator() when an RDD is cached. */ def getOrCompute[T]( rdd: RDD[T], - split: Partition, + partition: Partition, context: TaskContext, storageLevel: StorageLevel): Iterator[T] = { - val key = RDDBlockId(rdd.id, split.index) + val key = RDDBlockId(rdd.id, partition.index) logDebug(s"Looking for partition $key") blockManager.get(key) match { case Some(values) => @@ -46,79 +46,28 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) case None => - // Mark the split as loading (unless someone else marks it first) - loading.synchronized { - if (loading.contains(key)) { - logInfo(s"Another thread is loading $key, waiting for it to finish...") - while (loading.contains(key)) { - try { - loading.wait() - } catch { - case e: Exception => - logWarning(s"Got an exception while waiting for another thread to load $key", e) - } - } - logInfo(s"Finished waiting for $key") - /* See whether someone else has successfully loaded it. The main way this would fail - * is for the RDD-level cache eviction policy if someone else has loaded the same RDD - * partition but we didn't want to make space for it. However, that case is unlikely - * because it's unlikely that two threads would work on the same RDD partition. One - * downside of the current code is that threads wait serially if this does happen. */ - blockManager.get(key) match { - case Some(values) => - return new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) - case None => - logInfo(s"Whoever was loading $key failed; we'll try it ourselves") - loading.add(key) - } - } else { - loading.add(key) - } + // Acquire a lock for loading this partition + // If another thread already holds the lock, wait for it to finish return its results + val storedValues = acquireLockForPartition[T](key) + if (storedValues.isDefined) { + return new InterruptibleIterator[T](context, storedValues.get) } + + // Otherwise, we have to load the partition ourselves try { - // If we got here, we have to load the split logInfo(s"Partition $key not found, computing it") - val computedValues = rdd.computeOrReadCheckpoint(split, context) + val computedValues = rdd.computeOrReadCheckpoint(partition, context) - // Persist the result, so long as the task is not running locally + // If the task is running locally, do not persist the result if (context.runningLocally) { return computedValues } - // Keep track of blocks with updated statuses - var updatedBlocks = Seq[(BlockId, BlockStatus)]() - val returnValue: Iterator[T] = { - if (storageLevel.useDisk && !storageLevel.useMemory) { - /* In the case that this RDD is to be persisted using DISK_ONLY - * the iterator will be passed directly to the blockManager (rather then - * caching it to an ArrayBuffer first), then the resulting block data iterator - * will be passed back to the user. If the iterator generates a lot of data, - * this means that it doesn't all have to be held in memory at one time. - * This could also apply to MEMORY_ONLY_SER storage, but we need to make sure - * blocks aren't dropped by the block store before enabling that. */ - updatedBlocks = blockManager.put(key, computedValues, storageLevel, tellMaster = true) - blockManager.get(key) match { - case Some(values) => - values.asInstanceOf[Iterator[T]] - case None => - logInfo(s"Failure to store $key") - throw new SparkException("Block manager failed to return persisted value") - } - } else { - // In this case the RDD is cached to an array buffer. This will save the results - // if we're dealing with a 'one-time' iterator - val elements = new ArrayBuffer[Any] - elements ++= computedValues - updatedBlocks = blockManager.put(key, elements, storageLevel, tellMaster = true) - elements.iterator.asInstanceOf[Iterator[T]] - } - } - - // Update task metrics to include any blocks whose storage status is updated - val metrics = context.taskMetrics - metrics.updatedBlocks = Some(updatedBlocks) - - new InterruptibleIterator(context, returnValue) + // Otherwise, cache the values and keep track of any updates in block statuses + val updatedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + val cachedValues = putInBlockManager(key, computedValues, storageLevel, updatedBlocks) + context.taskMetrics.updatedBlocks = Some(updatedBlocks) + new InterruptibleIterator(context, cachedValues) } finally { loading.synchronized { @@ -128,4 +77,76 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { } } } + + /** + * Acquire a loading lock for the partition identified by the given block ID. + * + * If the lock is free, just acquire it and return None. Otherwise, another thread is already + * loading the partition, so we wait for it to finish and return the values loaded by the thread. + */ + private def acquireLockForPartition[T](id: RDDBlockId): Option[Iterator[T]] = { + loading.synchronized { + if (!loading.contains(id)) { + // If the partition is free, acquire its lock to compute its value + loading.add(id) + None + } else { + // Otherwise, wait for another thread to finish and return its result + logInfo(s"Another thread is loading $id, waiting for it to finish...") + while (loading.contains(id)) { + try { + loading.wait() + } catch { + case e: Exception => + logWarning(s"Exception while waiting for another thread to load $id", e) + } + } + logInfo(s"Finished waiting for $id") + val values = blockManager.get(id) + if (!values.isDefined) { + /* The block is not guaranteed to exist even after the other thread has finished. + * For instance, the block could be evicted after it was put, but before our get. + * In this case, we still need to load the partition ourselves. */ + logInfo(s"Whoever was loading $id failed; we'll try it ourselves") + loading.add(id) + } + values.map(_.asInstanceOf[Iterator[T]]) + } + } + } + + /** + * Cache the values of a partition, keeping track of any updates in the storage statuses + * of other blocks along the way. + */ + 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) + blockManager.get(key) match { + case Some(v) => v.asInstanceOf[Iterator[T]] + case None => + logInfo(s"Failure to store $key") + 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 + * 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]] + } + } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 0e8d551e4b2ab..bbf9f7388b074 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -17,11 +17,12 @@ package org.apache.spark.scheduler +import scala.language.existentials + import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashMap -import scala.language.existentials import org.apache.spark._ import org.apache.spark.rdd.{RDD, RDDCheckpointData} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 0098b5a59d1a5..859cdc524a581 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -25,10 +25,7 @@ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.rdd.{RDD, RDDCheckpointData} -import org.apache.spark.serializer.Serializer -import org.apache.spark.storage._ import org.apache.spark.shuffle.ShuffleWriter private[spark] object ShuffleMapTask { @@ -150,7 +147,7 @@ private[spark] class ShuffleMapTask( for (elem <- rdd.iterator(split, context)) { writer.write(elem.asInstanceOf[Product2[Any, Any]]) } - return writer.stop(success = true).get + writer.stop(success = true).get } catch { case e: Exception => if (writer != null) { 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 9a9be047c7245..b9b53b1a2f118 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala @@ -24,11 +24,11 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.Logging /** - * Abstract class to store blocks + * Abstract class to store blocks. */ -private[spark] -abstract class BlockStore(val blockManager: BlockManager) extends Logging { - def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel) : PutResult +private[spark] abstract class BlockStore(val blockManager: BlockManager) extends Logging { + + def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): PutResult /** * Put in a block and, possibly, also return its content as either bytes or another Iterator. @@ -37,11 +37,17 @@ abstract class BlockStore(val blockManager: BlockManager) extends Logging { * @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(blockId: BlockId, values: Iterator[Any], level: StorageLevel, - returnValues: Boolean) : PutResult + def putValues( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + returnValues: Boolean): PutResult - def putValues(blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel, - returnValues: Boolean) : PutResult + def putValues( + blockId: BlockId, + values: ArrayBuffer[Any], + level: StorageLevel, + returnValues: Boolean): PutResult /** * Return the size of a block in bytes. 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 084a566c48560..71f66c826c5b3 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -58,11 +58,11 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) val elements = new ArrayBuffer[Any] elements ++= values val sizeEstimate = SizeEstimator.estimate(elements.asInstanceOf[AnyRef]) - tryToPut(blockId, elements, sizeEstimate, true) - PutResult(sizeEstimate, Left(values.toIterator)) + val putAttempt = tryToPut(blockId, elements, sizeEstimate, deserialized = true) + PutResult(sizeEstimate, Left(values.toIterator), putAttempt.droppedBlocks) } else { - tryToPut(blockId, bytes, bytes.limit, false) - PutResult(bytes.limit(), Right(bytes.duplicate())) + val putAttempt = tryToPut(blockId, bytes, bytes.limit, deserialized = false) + PutResult(bytes.limit(), Right(bytes.duplicate()), putAttempt.droppedBlocks) } } From d4c7572dba1be49e55ceb38713652e5bcf485be8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 20 Jun 2014 17:16:56 -0700 Subject: [PATCH 003/124] Move ScriptTransformation into the appropriate place. Author: Reynold Xin Closes #1162 from rxin/script and squashes the following commits: 2c836b9 [Reynold Xin] Move ScriptTransformation into the appropriate place. --- .../spark/sql/hive/{ => execution}/ScriptTransformation.scala | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename sql/hive/src/main/scala/org/apache/spark/sql/hive/{ => execution}/ScriptTransformation.scala (100%) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala similarity index 100% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala From 2044784915554a890ca6f8450d8403495b2ee4f3 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Fri, 20 Jun 2014 17:55:54 -0700 Subject: [PATCH 004/124] [SQL] Use hive.SessionState, not the thread local SessionState Note that this is simply mimicing lookupRelation(). I do not have a concrete notion of why this solution is necessarily right-er than SessionState.get, but SessionState.get is returning null, which is bad. Author: Aaron Davidson Closes #1148 from aarondav/createtable and squashes the following commits: 37c3e7c [Aaron Davidson] [SQL] Use hive.SessionState, not the thread local SessionState --- .../scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index f923d68932f83..faa30c9ae5cca 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -105,7 +105,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with object CreateTables extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case InsertIntoCreatedTable(db, tableName, child) => - val databaseName = db.getOrElse(SessionState.get.getCurrentDatabase) + val databaseName = db.getOrElse(hive.sessionState.getCurrentDatabase) createTable(databaseName, tableName, child.output) From 08d0aca78c7ac060bf50b9b8fac71cbc47a56bbe Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Fri, 20 Jun 2014 18:25:33 -0700 Subject: [PATCH 005/124] SPARK-1902 Silence stacktrace from logs when doing port failover to port n+1 Before: ``` 14/06/08 23:58:23 WARN AbstractLifeCycle: FAILED SelectChannelConnector@0.0.0.0:4040: java.net.BindException: Address already in use java.net.BindException: Address already in use at sun.nio.ch.Net.bind0(Native Method) at sun.nio.ch.Net.bind(Net.java:444) at sun.nio.ch.Net.bind(Net.java:436) at sun.nio.ch.ServerSocketChannelImpl.bind(ServerSocketChannelImpl.java:214) at sun.nio.ch.ServerSocketAdaptor.bind(ServerSocketAdaptor.java:74) at org.eclipse.jetty.server.nio.SelectChannelConnector.open(SelectChannelConnector.java:187) at org.eclipse.jetty.server.AbstractConnector.doStart(AbstractConnector.java:316) at org.eclipse.jetty.server.nio.SelectChannelConnector.doStart(SelectChannelConnector.java:265) at org.eclipse.jetty.util.component.AbstractLifeCycle.start(AbstractLifeCycle.java:64) at org.eclipse.jetty.server.Server.doStart(Server.java:293) at org.eclipse.jetty.util.component.AbstractLifeCycle.start(AbstractLifeCycle.java:64) at org.apache.spark.ui.JettyUtils$$anonfun$1.apply$mcV$sp(JettyUtils.scala:192) at org.apache.spark.ui.JettyUtils$$anonfun$1.apply(JettyUtils.scala:192) at org.apache.spark.ui.JettyUtils$$anonfun$1.apply(JettyUtils.scala:192) at scala.util.Try$.apply(Try.scala:161) at org.apache.spark.ui.JettyUtils$.connect$1(JettyUtils.scala:191) at org.apache.spark.ui.JettyUtils$.startJettyServer(JettyUtils.scala:205) at org.apache.spark.ui.WebUI.bind(WebUI.scala:99) at org.apache.spark.SparkContext.(SparkContext.scala:223) at org.apache.spark.repl.SparkILoop.createSparkContext(SparkILoop.scala:957) at $line3.$read$$iwC$$iwC.(:8) at $line3.$read$$iwC.(:14) at $line3.$read.(:16) at $line3.$read$.(:20) at $line3.$read$.() at $line3.$eval$.(:7) at $line3.$eval$.() at $line3.$eval.$print() at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:788) at org.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1056) at org.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:614) at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:645) at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:609) at org.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:796) at org.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:841) at org.apache.spark.repl.SparkILoop.command(SparkILoop.scala:753) at org.apache.spark.repl.SparkILoopInit$$anonfun$initializeSpark$1.apply(SparkILoopInit.scala:121) at org.apache.spark.repl.SparkILoopInit$$anonfun$initializeSpark$1.apply(SparkILoopInit.scala:120) at org.apache.spark.repl.SparkIMain.beQuietDuring(SparkIMain.scala:263) at org.apache.spark.repl.SparkILoopInit$class.initializeSpark(SparkILoopInit.scala:120) at org.apache.spark.repl.SparkILoop.initializeSpark(SparkILoop.scala:56) at org.apache.spark.repl.SparkILoop$$anonfun$process$1$$anonfun$apply$mcZ$sp$5.apply$mcV$sp(SparkILoop.scala:913) at org.apache.spark.repl.SparkILoopInit$class.runThunks(SparkILoopInit.scala:142) at org.apache.spark.repl.SparkILoop.runThunks(SparkILoop.scala:56) at org.apache.spark.repl.SparkILoopInit$class.postInitialization(SparkILoopInit.scala:104) at org.apache.spark.repl.SparkILoop.postInitialization(SparkILoop.scala:56) at org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply$mcZ$sp(SparkILoop.scala:930) at org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply(SparkILoop.scala:884) at org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply(SparkILoop.scala:884) at scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135) at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:884) at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:982) at org.apache.spark.repl.Main$.main(Main.scala:31) at org.apache.spark.repl.Main.main(Main.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.apache.spark.deploy.SparkSubmit$.launch(SparkSubmit.scala:292) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:55) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) 14/06/08 23:58:23 WARN AbstractLifeCycle: FAILED org.eclipse.jetty.server.Server@7439e55a: java.net.BindException: Address already in use java.net.BindException: Address already in use at sun.nio.ch.Net.bind0(Native Method) at sun.nio.ch.Net.bind(Net.java:444) at sun.nio.ch.Net.bind(Net.java:436) at sun.nio.ch.ServerSocketChannelImpl.bind(ServerSocketChannelImpl.java:214) at sun.nio.ch.ServerSocketAdaptor.bind(ServerSocketAdaptor.java:74) at org.eclipse.jetty.server.nio.SelectChannelConnector.open(SelectChannelConnector.java:187) at org.eclipse.jetty.server.AbstractConnector.doStart(AbstractConnector.java:316) at org.eclipse.jetty.server.nio.SelectChannelConnector.doStart(SelectChannelConnector.java:265) at org.eclipse.jetty.util.component.AbstractLifeCycle.start(AbstractLifeCycle.java:64) at org.eclipse.jetty.server.Server.doStart(Server.java:293) at org.eclipse.jetty.util.component.AbstractLifeCycle.start(AbstractLifeCycle.java:64) at org.apache.spark.ui.JettyUtils$$anonfun$1.apply$mcV$sp(JettyUtils.scala:192) at org.apache.spark.ui.JettyUtils$$anonfun$1.apply(JettyUtils.scala:192) at org.apache.spark.ui.JettyUtils$$anonfun$1.apply(JettyUtils.scala:192) at scala.util.Try$.apply(Try.scala:161) at org.apache.spark.ui.JettyUtils$.connect$1(JettyUtils.scala:191) at org.apache.spark.ui.JettyUtils$.startJettyServer(JettyUtils.scala:205) at org.apache.spark.ui.WebUI.bind(WebUI.scala:99) at org.apache.spark.SparkContext.(SparkContext.scala:223) at org.apache.spark.repl.SparkILoop.createSparkContext(SparkILoop.scala:957) at $line3.$read$$iwC$$iwC.(:8) at $line3.$read$$iwC.(:14) at $line3.$read.(:16) at $line3.$read$.(:20) at $line3.$read$.() at $line3.$eval$.(:7) at $line3.$eval$.() at $line3.$eval.$print() at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:788) at org.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1056) at org.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:614) at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:645) at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:609) at org.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:796) at org.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:841) at org.apache.spark.repl.SparkILoop.command(SparkILoop.scala:753) at org.apache.spark.repl.SparkILoopInit$$anonfun$initializeSpark$1.apply(SparkILoopInit.scala:121) at org.apache.spark.repl.SparkILoopInit$$anonfun$initializeSpark$1.apply(SparkILoopInit.scala:120) at org.apache.spark.repl.SparkIMain.beQuietDuring(SparkIMain.scala:263) at org.apache.spark.repl.SparkILoopInit$class.initializeSpark(SparkILoopInit.scala:120) at org.apache.spark.repl.SparkILoop.initializeSpark(SparkILoop.scala:56) at org.apache.spark.repl.SparkILoop$$anonfun$process$1$$anonfun$apply$mcZ$sp$5.apply$mcV$sp(SparkILoop.scala:913) at org.apache.spark.repl.SparkILoopInit$class.runThunks(SparkILoopInit.scala:142) at org.apache.spark.repl.SparkILoop.runThunks(SparkILoop.scala:56) at org.apache.spark.repl.SparkILoopInit$class.postInitialization(SparkILoopInit.scala:104) at org.apache.spark.repl.SparkILoop.postInitialization(SparkILoop.scala:56) at org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply$mcZ$sp(SparkILoop.scala:930) at org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply(SparkILoop.scala:884) at org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply(SparkILoop.scala:884) at scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135) at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:884) at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:982) at org.apache.spark.repl.Main$.main(Main.scala:31) at org.apache.spark.repl.Main.main(Main.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.apache.spark.deploy.SparkSubmit$.launch(SparkSubmit.scala:292) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:55) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) 14/06/08 23:58:23 INFO JettyUtils: Failed to create UI at port, 4040. Trying again. 14/06/08 23:58:23 INFO JettyUtils: Error was: Failure(java.net.BindException: Address already in use) 14/06/08 23:58:23 INFO SparkUI: Started SparkUI at http://aash-mbp.local:4041 ```` After: ``` 14/06/09 00:04:12 INFO JettyUtils: Failed to create UI at port, 4040. Trying again. 14/06/09 00:04:12 INFO JettyUtils: Error was: Failure(java.net.BindException: Address already in use) 14/06/09 00:04:12 INFO Server: jetty-8.y.z-SNAPSHOT 14/06/09 00:04:12 INFO AbstractConnector: Started SelectChannelConnector@0.0.0.0:4041 14/06/09 00:04:12 INFO SparkUI: Started SparkUI at http://aash-mbp.local:4041 ``` Lengthy logging comes from this line of code in Jetty: http://grepcode.com/file/repo1.maven.org/maven2/org.eclipse.jetty.aggregate/jetty-all/9.1.3.v20140225/org/eclipse/jetty/util/component/AbstractLifeCycle.java#210 Author: Andrew Ash Closes #1019 from ash211/SPARK-1902 and squashes the following commits: 0dd02f7 [Andrew Ash] Leave old org.eclipse.jetty silencing in place 1e2866b [Andrew Ash] Address CR comments 9d85eed [Andrew Ash] SPARK-1902 Silence stacktrace from logs when doing port failover to port n+1 --- conf/log4j.properties.template | 1 + .../org/apache/spark/log4j-defaults.properties | 1 + .../org/apache/spark/executor/ExecutorBackend.scala | 2 +- .../main/scala/org/apache/spark/ui/JettyUtils.scala | 11 ++++++++--- 4 files changed, 11 insertions(+), 4 deletions(-) diff --git a/conf/log4j.properties.template b/conf/log4j.properties.template index f7f853559468a..89eec7d4b7f61 100644 --- a/conf/log4j.properties.template +++ b/conf/log4j.properties.template @@ -7,5 +7,6 @@ log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: # Settings to quiet third party logs that are too verbose log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties index f7f853559468a..89eec7d4b7f61 100644 --- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties +++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties @@ -7,5 +7,6 @@ log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: # Settings to quiet third party logs that are too verbose log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala index 3d34960653f5d..9da17428ed43d 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala @@ -26,4 +26,4 @@ import org.apache.spark.TaskState.TaskState */ private[spark] trait ExecutorBackend { def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) -} +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index b3ac2320f3431..a2535e3c1c41f 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -194,11 +194,16 @@ private[spark] object JettyUtils extends Logging { case s: Success[_] => (server, server.getConnectors.head.getLocalPort) case f: Failure[_] => + val nextPort = (currentPort + 1) % 65536 server.stop() pool.stop() - logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) - logInfo("Error was: " + f.toString) - connect((currentPort + 1) % 65536) + val msg = s"Failed to create UI on port $currentPort. Trying again on port $nextPort." + if (f.toString.contains("Address already in use")) { + logWarning(s"$msg - $f") + } else { + logError(msg, f.exception) + } + connect(nextPort) } } From e99903b84a530ebf97d46f4b2e1e78db0f65b041 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Fri, 20 Jun 2014 18:42:02 -0700 Subject: [PATCH 006/124] [SPARK-1970] Update unit test in XORShiftRandomSuite to use ChiSquareTest from commons-math3 Updating the chisquare unit test in XORShiftRandomSuite to use the ChiSquareTest in commons-math3 instead of hardcoding the chisquare statistic for the desired confidence interval. Author: Doris Xin Closes #1073 from dorx/math3Unit and squashes the following commits: da0e891 [Doris Xin] remove math3 from common pom 9954143 [Doris Xin] merge master c19948f [Doris Xin] Merge branch 'master' into math3Unit 8f84f19 [Doris Xin] [SPARK-1970] unit test in XORShiftRandomSuite ffea61a [Doris Xin] SPARK-1939: Refactor takeSample method in RDD 1441977 [Doris Xin] SPARK-1939 Refactor takeSample method in RDD to use ScaSRS --- .../util/random/XORShiftRandomSuite.scala | 49 +++++++------------ 1 file changed, 18 insertions(+), 31 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index e15fd59a5a8bb..ef7178bcdf5c2 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.util.random import org.scalatest.FunSuite import org.scalatest.Matchers +import org.apache.commons.math3.stat.inference.ChiSquareTest + import org.apache.spark.util.Utils.times import scala.language.reflectiveCalls @@ -33,45 +35,30 @@ class XORShiftRandomSuite extends FunSuite with Matchers { } /* - * This test is based on a chi-squared test for randomness. The values are hard-coded - * so as not to create Spark's dependency on apache.commons.math3 just to call one - * method for calculating the exact p-value for a given number of random numbers - * and bins. In case one would want to move to a full-fledged test based on - * apache.commons.math3, the relevant class is here: - * org.apache.commons.math3.stat.inference.ChiSquareTest + * This test is based on a chi-squared test for randomness. */ test ("XORShift generates valid random numbers") { val f = fixture - val numBins = 10 - // create 10 bins - val bins = Array.fill(numBins)(0) + val numBins = 10 // create 10 bins + val numRows = 5 // create 5 rows + val bins = Array.ofDim[Long](numRows, numBins) - // populate bins based on modulus of the random number - times(f.hundMil) {bins(math.abs(f.xorRand.nextInt) % 10) += 1} + // populate bins based on modulus of the random number for each row + for (r <- 0 to numRows-1) { + times(f.hundMil) {bins(r)(math.abs(f.xorRand.nextInt) % numBins) += 1} + } - /* since the seed is deterministic, until the algorithm is changed, we know the result will be - * exactly this: Array(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, - * 10000790, 10002286, 9998699), so the test will never fail at the prespecified (5%) - * significance level. However, should the RNG implementation change, the test should still - * pass at the same significance level. The chi-squared test done in R gave the following - * results: - * > chisq.test(c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, - * 10000790, 10002286, 9998699)) - * Chi-squared test for given probabilities - * data: c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, 10000790, - * 10002286, 9998699) - * X-squared = 11.975, df = 9, p-value = 0.2147 - * Note that the p-value was ~0.22. The test will fail if alpha < 0.05, which for 100 million - * random numbers - * and 10 bins will happen at X-squared of ~16.9196. So, the test will fail if X-squared - * is greater than or equal to that number. + /* + * Perform the chi square test on the 5 rows of randomly generated numbers evenly divided into + * 10 bins. chiSquareTest returns true iff the null hypothesis (that the classifications + * represented by the counts in the columns of the input 2-way table are independent of the + * rows) can be rejected with 100 * (1 - alpha) percent confidence, where alpha is prespeficied + * as 0.05 */ - val binSize = f.hundMil/numBins - val xSquared = bins.map(x => math.pow((binSize - x), 2)/binSize).sum - xSquared should be < (16.9196) - + val chiTest = new ChiSquareTest + assert(chiTest.chiSquareTest(bins, 0.05) === false) } test ("XORShift with zero seed") { From a6786424954218fc31d8cc638a5c7d6567b55047 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 20 Jun 2014 18:44:54 -0700 Subject: [PATCH 007/124] HOTFIX: Fixing style error introduced by 08d0ac --- .../main/scala/org/apache/spark/executor/ExecutorBackend.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala index 9da17428ed43d..e07cb31cbe4ba 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala @@ -26,4 +26,5 @@ import org.apache.spark.TaskState.TaskState */ private[spark] trait ExecutorBackend { def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) -} \ No newline at end of file +} + From 010c460d627c1917dc47b09e59fd41172bbf90b3 Mon Sep 17 00:00:00 2001 From: Anant Date: Fri, 20 Jun 2014 18:54:00 -0700 Subject: [PATCH 008/124] [SPARK-2061] Made splits deprecated in JavaRDDLike The jira for the issue can be found at: https://issues.apache.org/jira/browse/SPARK-2061 Most of spark has used over to consistently using `partitions` instead of `splits`. We should do likewise and add a `partitions` method to JavaRDDLike and have `splits` just call that. We should also go through all cases where other API's (e.g. Python) call `splits` and we should change those to use the newer API. Author: Anant Closes #1062 from anantasty/SPARK-2061 and squashes the following commits: b83ce6b [Anant] Fixed syntax issue 21f9210 [Anant] Fixed version number in deprecation string 9315b76 [Anant] made related changes to use partitions in python api 8c62dd1 [Anant] Made splits deprecated in JavaRDDLike --- .../main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 5 ++++- core/src/test/java/org/apache/spark/JavaAPISuite.java | 2 +- python/pyspark/context.py | 2 +- python/pyspark/rdd.py | 4 ++-- 4 files changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 330569a8d8837..f917cfd1419ec 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -43,8 +43,11 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def rdd: RDD[T] - /** Set of partitions in this RDD. */ + @deprecated("Use partitions() instead.", "1.1.0") def splits: JList[Partition] = new java.util.ArrayList(rdd.partitions.toSeq) + + /** Set of partitions in this RDD. */ + def partitions: JList[Partition] = new java.util.ArrayList(rdd.partitions.toSeq) /** The [[org.apache.spark.SparkContext]] that this RDD was created on. */ def context: SparkContext = rdd.context diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 761f2d6a77d33..1d7a7be6cfeb3 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -741,7 +741,7 @@ public void persist() { public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); TaskContext context = new TaskContext(0, 0, 0, false, new TaskMetrics()); - Assert.assertEquals(1, rdd.iterator(rdd.splits().get(0), context).next().intValue()); + Assert.assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue()); } @Test diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 062bec2381a8f..95c54e7a5ad63 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -704,7 +704,7 @@ def runJob(self, rdd, partitionFunc, partitions = None, allowLocal = False): [0, 1, 16, 25] """ if partitions == None: - partitions = range(rdd._jrdd.splits().size()) + partitions = range(rdd._jrdd.partitions().size()) javaPartitions = ListConverter().convert(partitions, self._gateway._gateway_client) # Implementation note: This is implemented as a mapPartitions followed diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 1d55c35a8bf48..f64f48e3a4c9c 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -321,7 +321,7 @@ def getNumPartitions(self): >>> rdd.getNumPartitions() 2 """ - return self._jrdd.splits().size() + return self._jrdd.partitions().size() def filter(self, f): """ @@ -922,7 +922,7 @@ def take(self, num): [91, 92, 93] """ items = [] - totalParts = self._jrdd.splits().size() + totalParts = self._jrdd.partitions().size() partsScanned = 0 while len(items) < num and partsScanned < totalParts: From 648553d48ee1f830406750b50ec4cc322bcf47fe Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 20 Jun 2014 20:05:12 -0700 Subject: [PATCH 009/124] Fix some tests. - JavaAPISuite was trying to compare a bare path with a URI. Fix by extracting the path from the URI, since we know it should be a local path anyway/ - b9be1609 excluded the ASM dependency everywhere, but easymock needs it (because cglib needs it). So re-add the dependency, with test scope this time. The second one above actually uncovered a weird situation: the maven test target works, even though I can't find the class sbt complains about in its classpath. sbt complains with: [error] Uncaught exception when running org.apache.spark.util .random.RandomSamplerSuite: java.lang.NoClassDefFoundError: org/objectweb/asm/Type To avoid more weirdness caused by that, I explicitly added the asm dependency to both maven and sbt (for tests only), and verified the classes don't end up in the final assembly. Author: Marcelo Vanzin Closes #917 from vanzin/flaky-tests and squashes the following commits: d022320 [Marcelo Vanzin] Fix some tests. --- core/pom.xml | 5 +++++ core/src/test/java/org/apache/spark/JavaAPISuite.java | 5 +++-- pom.xml | 7 +++++++ project/SparkBuild.scala | 10 ++++++---- 4 files changed, 21 insertions(+), 6 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index bd6767e03bb9d..8c23842730e37 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -244,6 +244,11 @@ easymockclassextension test + + asm + asm + test + com.novocode junit-interface diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 1d7a7be6cfeb3..b2868b59ce6c6 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -18,6 +18,7 @@ package org.apache.spark; import java.io.*; +import java.net.URI; import java.util.*; import scala.Tuple2; @@ -768,7 +769,7 @@ public void textFiles() throws IOException { } @Test - public void wholeTextFiles() throws IOException { + public void wholeTextFiles() throws Exception { byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); byte[] content2 = "spark is also easy to use.\n".getBytes("utf-8"); @@ -784,7 +785,7 @@ public void wholeTextFiles() throws IOException { List> result = readRDD.collect(); for (Tuple2 res : result) { - Assert.assertEquals(res._2(), container.get(res._1())); + Assert.assertEquals(res._2(), container.get(new URI(res._1()).getPath())); } } diff --git a/pom.xml b/pom.xml index 0d46bb4114f73..05f76d566e9d1 100644 --- a/pom.xml +++ b/pom.xml @@ -468,6 +468,13 @@ 3.1 test + + + asm + asm + 3.3.1 + test + org.mockito mockito-all diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 7bb39dc77120b..55a2aa0fc7141 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -293,7 +293,9 @@ object SparkBuild extends Build { "com.novocode" % "junit-interface" % "0.10" % "test", "org.easymock" % "easymockclassextension" % "3.1" % "test", "org.mockito" % "mockito-all" % "1.9.0" % "test", - "junit" % "junit" % "4.10" % "test" + "junit" % "junit" % "4.10" % "test", + // Needed by cglib which is needed by easymock. + "asm" % "asm" % "3.3.1" % "test" ), testOptions += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), @@ -461,7 +463,7 @@ object SparkBuild extends Build { def toolsSettings = sharedSettings ++ Seq( name := "spark-tools", - libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v ), + libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v), libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-reflect" % v ) ) ++ assemblySettings ++ extraAssemblySettings @@ -630,9 +632,9 @@ object SparkBuild extends Build { scalaVersion := "2.10.4", retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", - libraryDependencies := Seq("spark-streaming-mqtt", "spark-streaming-zeromq", + libraryDependencies := Seq("spark-streaming-mqtt", "spark-streaming-zeromq", "spark-streaming-flume", "spark-streaming-kafka", "spark-streaming-twitter", - "spark-streaming", "spark-mllib", "spark-bagel", "spark-graphx", + "spark-streaming", "spark-mllib", "spark-bagel", "spark-graphx", "spark-core").map(sparkPreviousArtifact(_).get intransitive()) ) From ca5d8b5904dc6dd5b691af506d3a842e508b3673 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 20 Jun 2014 22:49:48 -0700 Subject: [PATCH 010/124] [SQL] Pass SQLContext instead of SparkContext into physical operators. This makes it easier to use config options in operators. Author: Reynold Xin Closes #1164 from rxin/sqlcontext and squashes the following commits: 797b2fd [Reynold Xin] Pass SQLContext instead of SparkContext into physical operators. --- .../org/apache/spark/sql/SQLContext.scala | 4 +++- .../spark/sql/execution/Aggregate.scala | 5 +++-- .../spark/sql/execution/SparkStrategies.scala | 22 +++++++++---------- .../spark/sql/execution/basicOperators.scala | 20 +++++++++-------- .../apache/spark/sql/execution/joins.scala | 21 +++++++++--------- .../sql/parquet/ParquetTableOperations.scala | 21 +++++++++--------- .../spark/sql/parquet/ParquetQuerySuite.scala | 2 +- 7 files changed, 51 insertions(+), 44 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index ab376e5504d35..c60af28b2a1f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -221,7 +221,9 @@ class SQLContext(@transient val sparkContext: SparkContext) } protected[sql] class SparkPlanner extends SparkStrategies { - val sparkContext = self.sparkContext + val sparkContext: SparkContext = self.sparkContext + + val sqlContext: SQLContext = self def numPartitions = self.numShufflePartitions diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala index 34d88fe4bd7de..d85d2d7844e0b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala @@ -24,6 +24,7 @@ import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.SQLContext /** * :: DeveloperApi :: @@ -41,7 +42,7 @@ case class Aggregate( partial: Boolean, groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], - child: SparkPlan)(@transient sc: SparkContext) + child: SparkPlan)(@transient sqlContext: SQLContext) extends UnaryNode with NoBind { override def requiredChildDistribution = @@ -55,7 +56,7 @@ case class Aggregate( } } - override def otherCopyArgs = sc :: Nil + override def otherCopyArgs = sqlContext :: Nil // HACK: Generators don't correctly preserve their output through serializations so we grab // out child's output attributes statically here. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 4694f25d6d630..bd8ae4cddef89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -40,7 +40,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // no predicate can be evaluated by matching hash keys case logical.Join(left, right, LeftSemi, condition) => execution.LeftSemiJoinBNL( - planLater(left), planLater(right), condition)(sparkContext) :: Nil + planLater(left), planLater(right), condition)(sqlContext) :: Nil case _ => Nil } } @@ -103,7 +103,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { partial = true, groupingExpressions, partialComputation, - planLater(child))(sparkContext))(sparkContext) :: Nil + planLater(child))(sqlContext))(sqlContext) :: Nil } else { Nil } @@ -115,7 +115,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Join(left, right, joinType, condition) => execution.BroadcastNestedLoopJoin( - planLater(left), planLater(right), joinType, condition)(sparkContext) :: Nil + planLater(left), planLater(right), joinType, condition)(sqlContext) :: Nil case _ => Nil } } @@ -143,7 +143,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object TakeOrdered extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Limit(IntegerLiteral(limit), logical.Sort(order, child)) => - execution.TakeOrdered(limit, order, planLater(child))(sparkContext) :: Nil + execution.TakeOrdered(limit, order, planLater(child))(sqlContext) :: Nil case _ => Nil } } @@ -155,9 +155,9 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { val relation = ParquetRelation.create(path, child, sparkContext.hadoopConfiguration) // Note: overwrite=false because otherwise the metadata we just created will be deleted - InsertIntoParquetTable(relation, planLater(child), overwrite=false)(sparkContext) :: Nil + InsertIntoParquetTable(relation, planLater(child), overwrite=false)(sqlContext) :: Nil case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => - InsertIntoParquetTable(table, planLater(child), overwrite)(sparkContext) :: Nil + InsertIntoParquetTable(table, planLater(child), overwrite)(sqlContext) :: Nil case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) => val prunePushedDownFilters = if (sparkContext.conf.getBoolean(ParquetFilters.PARQUET_FILTER_PUSHDOWN_ENABLED, true)) { @@ -186,7 +186,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { projectList, filters, prunePushedDownFilters, - ParquetTableScan(_, relation, filters)(sparkContext)) :: Nil + ParquetTableScan(_, relation, filters)(sqlContext)) :: Nil case _ => Nil } @@ -211,7 +211,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Distinct(child) => execution.Aggregate( - partial = false, child.output, child.output, planLater(child))(sparkContext) :: Nil + partial = false, child.output, child.output, planLater(child))(sqlContext) :: Nil case logical.Sort(sortExprs, child) => // This sort is a global sort. Its requiredDistribution will be an OrderedDistribution. execution.Sort(sortExprs, global = true, planLater(child)):: Nil @@ -224,7 +224,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.Filter(condition, child) => execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => - execution.Aggregate(partial = false, group, agg, planLater(child))(sparkContext) :: Nil + execution.Aggregate(partial = false, group, agg, planLater(child))(sqlContext) :: Nil case logical.Sample(fraction, withReplacement, seed, child) => execution.Sample(fraction, withReplacement, seed, planLater(child)) :: Nil case logical.LocalRelation(output, data) => @@ -233,9 +233,9 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { new GenericRow(r.productIterator.map(convertToCatalyst).toArray): Row)) execution.ExistingRdd(output, dataAsRdd) :: Nil case logical.Limit(IntegerLiteral(limit), child) => - execution.Limit(limit, planLater(child))(sparkContext) :: Nil + execution.Limit(limit, planLater(child))(sqlContext) :: Nil case Unions(unionChildren) => - execution.Union(unionChildren.map(planLater))(sparkContext) :: Nil + execution.Union(unionChildren.map(planLater))(sqlContext) :: Nil case logical.Generate(generator, join, outer, _, child) => execution.Generate(generator, join = join, outer = outer, planLater(child)) :: Nil case logical.NoRelation => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 8969794c69933..18f4a5877bb21 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -20,8 +20,9 @@ package org.apache.spark.sql.execution import scala.reflect.runtime.universe.TypeTag import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.{HashPartitioner, SparkConf, SparkContext} +import org.apache.spark.{HashPartitioner, SparkConf} import org.apache.spark.rdd.{RDD, ShuffledRDD} +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ @@ -70,12 +71,12 @@ case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: * :: DeveloperApi :: */ @DeveloperApi -case class Union(children: Seq[SparkPlan])(@transient sc: SparkContext) extends SparkPlan { +case class Union(children: Seq[SparkPlan])(@transient sqlContext: SQLContext) extends SparkPlan { // TODO: attributes output by union should be distinct for nullability purposes override def output = children.head.output - override def execute() = sc.union(children.map(_.execute())) + override def execute() = sqlContext.sparkContext.union(children.map(_.execute())) - override def otherCopyArgs = sc :: Nil + override def otherCopyArgs = sqlContext :: Nil } /** @@ -87,11 +88,12 @@ case class Union(children: Seq[SparkPlan])(@transient sc: SparkContext) extends * data to a single partition to compute the global limit. */ @DeveloperApi -case class Limit(limit: Int, child: SparkPlan)(@transient sc: SparkContext) extends UnaryNode { +case class Limit(limit: Int, child: SparkPlan)(@transient sqlContext: SQLContext) + extends UnaryNode { // TODO: Implement a partition local limit, and use a strategy to generate the proper limit plan: // partition local limit -> exchange into one partition -> partition local limit again - override def otherCopyArgs = sc :: Nil + override def otherCopyArgs = sqlContext :: Nil override def output = child.output @@ -117,8 +119,8 @@ case class Limit(limit: Int, child: SparkPlan)(@transient sc: SparkContext) exte */ @DeveloperApi case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) - (@transient sc: SparkContext) extends UnaryNode { - override def otherCopyArgs = sc :: Nil + (@transient sqlContext: SQLContext) extends UnaryNode { + override def otherCopyArgs = sqlContext :: Nil override def output = child.output @@ -129,7 +131,7 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. - override def execute() = sc.makeRDD(executeCollect(), 1) + override def execute() = sqlContext.sparkContext.makeRDD(executeCollect(), 1) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index 8d7a5ba59f96a..84bdde38b7e9e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -19,9 +19,8 @@ package org.apache.spark.sql.execution import scala.collection.mutable.{ArrayBuffer, BitSet} -import org.apache.spark.SparkContext - import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning} @@ -200,13 +199,13 @@ case class LeftSemiJoinHash( @DeveloperApi case class LeftSemiJoinBNL( streamed: SparkPlan, broadcast: SparkPlan, condition: Option[Expression]) - (@transient sc: SparkContext) + (@transient sqlContext: SQLContext) extends BinaryNode { // TODO: Override requiredChildDistribution. override def outputPartitioning: Partitioning = streamed.outputPartitioning - override def otherCopyArgs = sc :: Nil + override def otherCopyArgs = sqlContext :: Nil def output = left.output @@ -223,7 +222,8 @@ case class LeftSemiJoinBNL( def execute() = { - val broadcastedRelation = sc.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) + val broadcastedRelation = + sqlContext.sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) streamed.execute().mapPartitions { streamedIter => val joinedRow = new JoinedRow @@ -263,13 +263,13 @@ case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNod @DeveloperApi case class BroadcastNestedLoopJoin( streamed: SparkPlan, broadcast: SparkPlan, joinType: JoinType, condition: Option[Expression]) - (@transient sc: SparkContext) + (@transient sqlContext: SQLContext) extends BinaryNode { // TODO: Override requiredChildDistribution. override def outputPartitioning: Partitioning = streamed.outputPartitioning - override def otherCopyArgs = sc :: Nil + override def otherCopyArgs = sqlContext :: Nil def output = left.output ++ right.output @@ -286,7 +286,8 @@ case class BroadcastNestedLoopJoin( def execute() = { - val broadcastedRelation = sc.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) + val broadcastedRelation = + sqlContext.sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) val streamedPlusMatches = streamed.execute().mapPartitions { streamedIter => val matchedRows = new ArrayBuffer[Row] @@ -337,7 +338,7 @@ case class BroadcastNestedLoopJoin( } // TODO: Breaks lineage. - sc.union( - streamedPlusMatches.flatMap(_._1), sc.makeRDD(rightOuterMatches)) + sqlContext.sparkContext.union( + streamedPlusMatches.flatMap(_._1), sqlContext.sparkContext.makeRDD(rightOuterMatches)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 624f2e2fa13f6..ade823b51c9cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -33,10 +33,10 @@ import parquet.hadoop.util.ContextUtil import parquet.io.InvalidRecordException import parquet.schema.MessageType -import org.apache.spark.{Logging, SerializableWritable, SparkContext, TaskContext} +import org.apache.spark.{Logging, SerializableWritable, TaskContext} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row} -import org.apache.spark.sql.catalyst.types.StructType import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode} /** @@ -49,10 +49,11 @@ case class ParquetTableScan( output: Seq[Attribute], relation: ParquetRelation, columnPruningPred: Seq[Expression])( - @transient val sc: SparkContext) + @transient val sqlContext: SQLContext) extends LeafNode { override def execute(): RDD[Row] = { + val sc = sqlContext.sparkContext val job = new Job(sc.hadoopConfiguration) ParquetInputFormat.setReadSupportClass( job, @@ -93,7 +94,7 @@ case class ParquetTableScan( .filter(_ != null) // Parquet's record filters may produce null values } - override def otherCopyArgs = sc :: Nil + override def otherCopyArgs = sqlContext :: Nil /** * Applies a (candidate) projection. @@ -104,7 +105,7 @@ case class ParquetTableScan( def pruneColumns(prunedAttributes: Seq[Attribute]): ParquetTableScan = { val success = validateProjection(prunedAttributes) if (success) { - ParquetTableScan(prunedAttributes, relation, columnPruningPred)(sc) + ParquetTableScan(prunedAttributes, relation, columnPruningPred)(sqlContext) } else { sys.error("Warning: Could not validate Parquet schema projection in pruneColumns") this @@ -152,7 +153,7 @@ case class InsertIntoParquetTable( relation: ParquetRelation, child: SparkPlan, overwrite: Boolean = false)( - @transient val sc: SparkContext) + @transient val sqlContext: SQLContext) extends UnaryNode with SparkHadoopMapReduceUtil { /** @@ -168,7 +169,7 @@ case class InsertIntoParquetTable( val childRdd = child.execute() assert(childRdd != null) - val job = new Job(sc.hadoopConfiguration) + val job = new Job(sqlContext.sparkContext.hadoopConfiguration) val writeSupport = if (child.output.map(_.dataType).forall(_.isPrimitive)) { @@ -204,7 +205,7 @@ case class InsertIntoParquetTable( override def output = child.output - override def otherCopyArgs = sc :: Nil + override def otherCopyArgs = sqlContext :: Nil /** * Stores the given Row RDD as a Hadoop file. @@ -231,7 +232,7 @@ case class InsertIntoParquetTable( val wrappedConf = new SerializableWritable(job.getConfiguration) val formatter = new SimpleDateFormat("yyyyMMddHHmm") val jobtrackerID = formatter.format(new Date()) - val stageId = sc.newRddId() + val stageId = sqlContext.sparkContext.newRddId() val taskIdOffset = if (overwrite) { @@ -270,7 +271,7 @@ case class InsertIntoParquetTable( val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId) val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext) jobCommitter.setupJob(jobTaskContext) - sc.runJob(rdd, writeShard _) + sqlContext.sparkContext.runJob(rdd, writeShard _) jobCommitter.commitJob(jobTaskContext) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 7714eb1b5628a..2ca0c1cdcbeca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -166,7 +166,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val scanner = new ParquetTableScan( ParquetTestData.testData.output, ParquetTestData.testData, - Seq())(TestSQLContext.sparkContext) + Seq())(TestSQLContext) val projected = scanner.pruneColumns(ParquetTypesConverter .convertToAttributes(MessageTypeParser .parseMessageType(ParquetTestData.subTestSchema))) From ec935abce13b60f353236566da149c0c87bb1002 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 21 Jun 2014 12:04:18 -0700 Subject: [PATCH 011/124] [SQL] Break hiveOperators.scala into multiple files. The single file was getting very long (500+ loc). Author: Reynold Xin Closes #1166 from rxin/hiveOperators and squashes the following commits: 5b43068 [Reynold Xin] [SQL] Break hiveOperators.scala into multiple files. --- .../spark/sql/hive/HiveStrategies.scala | 9 +- .../execution/DescribeHiveTableCommand.scala | 88 +++ .../sql/hive/execution/HiveTableScan.scala | 223 ++++++++ .../hive/execution/InsertIntoHiveTable.scala | 248 +++++++++ .../sql/hive/execution/NativeCommand.scala | 47 ++ .../sql/hive/execution/hiveOperators.scala | 524 ------------------ 6 files changed, 610 insertions(+), 529 deletions(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala delete mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/hiveOperators.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index af7687b40429b..4d0fab4140b21 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -64,7 +64,6 @@ private[hive] trait HiveStrategies { val partitionKeyIds = relation.partitionKeys.map(_.exprId).toSet val (pruningPredicates, otherPredicates) = predicates.partition { _.references.map(_.exprId).subsetOf(partitionKeyIds) - } pruneFilterProject( @@ -81,16 +80,16 @@ private[hive] trait HiveStrategies { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.NativeCommand(sql) => NativeCommand(sql, plan.output)(context) :: Nil - case describe: logical.DescribeCommand => { + + case describe: logical.DescribeCommand => val resolvedTable = context.executePlan(describe.table).analyzed resolvedTable match { case t: MetastoreRelation => - Seq(DescribeHiveTableCommand( - t, describe.output, describe.isExtended)(context)) + Seq(DescribeHiveTableCommand(t, describe.output, describe.isExtended)(context)) case o: LogicalPlan => Seq(DescribeCommand(planLater(o), describe.output)(context)) } - } + case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala new file mode 100644 index 0000000000000..a40e89e0d382b --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala @@ -0,0 +1,88 @@ +/* + * 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.execution + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.hive.metastore.api.FieldSchema + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow, Row} +import org.apache.spark.sql.execution.{Command, LeafNode} +import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation} + +/** + * Implementation for "describe [extended] table". + * + * :: DeveloperApi :: + */ +@DeveloperApi +case class DescribeHiveTableCommand( + table: MetastoreRelation, + output: Seq[Attribute], + isExtended: Boolean)( + @transient context: HiveContext) + extends LeafNode with Command { + + // Strings with the format like Hive. It is used for result comparison in our unit tests. + lazy val hiveString: Seq[String] = { + val alignment = 20 + val delim = "\t" + + sideEffectResult.map { + case (name, dataType, comment) => + String.format("%-" + alignment + "s", name) + delim + + String.format("%-" + alignment + "s", dataType) + delim + + String.format("%-" + alignment + "s", Option(comment).getOrElse("None")) + } + } + + override protected[sql] lazy val sideEffectResult: Seq[(String, String, String)] = { + // Trying to mimic the format of Hive's output. But not exactly the same. + var results: Seq[(String, String, String)] = Nil + + val columns: Seq[FieldSchema] = table.hiveQlTable.getCols + val partitionColumns: Seq[FieldSchema] = table.hiveQlTable.getPartCols + results ++= columns.map(field => (field.getName, field.getType, field.getComment)) + if (!partitionColumns.isEmpty) { + val partColumnInfo = + partitionColumns.map(field => (field.getName, field.getType, field.getComment)) + results ++= + partColumnInfo ++ + Seq(("# Partition Information", "", "")) ++ + Seq((s"# ${output.get(0).name}", output.get(1).name, output.get(2).name)) ++ + partColumnInfo + } + + if (isExtended) { + results ++= Seq(("Detailed Table Information", table.hiveQlTable.getTTable.toString, "")) + } + + results + } + + override def execute(): RDD[Row] = { + val rows = sideEffectResult.map { + case (name, dataType, comment) => new GenericRow(Array[Any](name, dataType, comment)) + } + context.sparkContext.parallelize(rows, 1) + } + + override def otherCopyArgs = context :: Nil +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala new file mode 100644 index 0000000000000..ef8bae74530ec --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -0,0 +1,223 @@ +/* + * 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.execution + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition} +import org.apache.hadoop.hive.serde.serdeConstants +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils +import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption +import org.apache.hadoop.hive.serde2.objectinspector.primitive._ +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types.{BooleanType, DataType} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.hive._ +import org.apache.spark.util.MutablePair + +/** + * :: DeveloperApi :: + * The Hive table scan operator. Column and partition pruning are both handled. + * + * @param attributes Attributes to be fetched from the Hive table. + * @param relation The Hive table be be scanned. + * @param partitionPruningPred An optional partition pruning predicate for partitioned table. + */ +@DeveloperApi +case class HiveTableScan( + attributes: Seq[Attribute], + relation: MetastoreRelation, + partitionPruningPred: Option[Expression])( + @transient val context: HiveContext) + extends LeafNode + with HiveInspectors { + + require(partitionPruningPred.isEmpty || relation.hiveQlTable.isPartitioned, + "Partition pruning predicates only supported for partitioned tables.") + + // Bind all partition key attribute references in the partition pruning predicate for later + // evaluation. + private[this] val boundPruningPred = partitionPruningPred.map { pred => + require( + pred.dataType == BooleanType, + s"Data type of predicate $pred must be BooleanType rather than ${pred.dataType}.") + + BindReferences.bindReference(pred, relation.partitionKeys) + } + + @transient + private[this] val hadoopReader = new HadoopTableReader(relation.tableDesc, context) + + /** + * The hive object inspector for this table, which can be used to extract values from the + * serialized row representation. + */ + @transient + private[this] lazy val objectInspector = + relation.tableDesc.getDeserializer.getObjectInspector.asInstanceOf[StructObjectInspector] + + /** + * Functions that extract the requested attributes from the hive output. Partitioned values are + * casted from string to its declared data type. + */ + @transient + protected lazy val attributeFunctions: Seq[(Any, Array[String]) => Any] = { + attributes.map { a => + val ordinal = relation.partitionKeys.indexOf(a) + if (ordinal >= 0) { + val dataType = relation.partitionKeys(ordinal).dataType + (_: Any, partitionKeys: Array[String]) => { + castFromString(partitionKeys(ordinal), dataType) + } + } else { + val ref = objectInspector.getAllStructFieldRefs + .find(_.getFieldName == a.name) + .getOrElse(sys.error(s"Can't find attribute $a")) + val fieldObjectInspector = ref.getFieldObjectInspector + + val unwrapHiveData = fieldObjectInspector match { + case _: HiveVarcharObjectInspector => + (value: Any) => value.asInstanceOf[HiveVarchar].getValue + case _: HiveDecimalObjectInspector => + (value: Any) => BigDecimal(value.asInstanceOf[HiveDecimal].bigDecimalValue()) + case _ => + identity[Any] _ + } + + (row: Any, _: Array[String]) => { + val data = objectInspector.getStructFieldData(row, ref) + val hiveData = unwrapData(data, fieldObjectInspector) + if (hiveData != null) unwrapHiveData(hiveData) else null + } + } + } + } + + private[this] def castFromString(value: String, dataType: DataType) = { + Cast(Literal(value), dataType).eval(null) + } + + private def addColumnMetadataToConf(hiveConf: HiveConf) { + // Specifies IDs and internal names of columns to be scanned. + val neededColumnIDs = attributes.map(a => relation.output.indexWhere(_.name == a.name): Integer) + val columnInternalNames = neededColumnIDs.map(HiveConf.getColumnInternalName(_)).mkString(",") + + if (attributes.size == relation.output.size) { + ColumnProjectionUtils.setFullyReadColumns(hiveConf) + } else { + ColumnProjectionUtils.appendReadColumnIDs(hiveConf, neededColumnIDs) + } + + ColumnProjectionUtils.appendReadColumnNames(hiveConf, attributes.map(_.name)) + + // Specifies types and object inspectors of columns to be scanned. + val structOI = ObjectInspectorUtils + .getStandardObjectInspector( + relation.tableDesc.getDeserializer.getObjectInspector, + ObjectInspectorCopyOption.JAVA) + .asInstanceOf[StructObjectInspector] + + val columnTypeNames = structOI + .getAllStructFieldRefs + .map(_.getFieldObjectInspector) + .map(TypeInfoUtils.getTypeInfoFromObjectInspector(_).getTypeName) + .mkString(",") + + hiveConf.set(serdeConstants.LIST_COLUMN_TYPES, columnTypeNames) + hiveConf.set(serdeConstants.LIST_COLUMNS, columnInternalNames) + } + + addColumnMetadataToConf(context.hiveconf) + + private def inputRdd = if (!relation.hiveQlTable.isPartitioned) { + hadoopReader.makeRDDForTable(relation.hiveQlTable) + } else { + hadoopReader.makeRDDForPartitionedTable(prunePartitions(relation.hiveQlPartitions)) + } + + /** + * Prunes partitions not involve the query plan. + * + * @param partitions All partitions of the relation. + * @return Partitions that are involved in the query plan. + */ + private[hive] def prunePartitions(partitions: Seq[HivePartition]) = { + boundPruningPred match { + case None => partitions + case Some(shouldKeep) => partitions.filter { part => + val dataTypes = relation.partitionKeys.map(_.dataType) + val castedValues = for ((value, dataType) <- part.getValues.zip(dataTypes)) yield { + castFromString(value, dataType) + } + + // Only partitioned values are needed here, since the predicate has already been bound to + // partition key attribute references. + val row = new GenericRow(castedValues.toArray) + shouldKeep.eval(row).asInstanceOf[Boolean] + } + } + } + + override def execute() = { + inputRdd.mapPartitions { iterator => + if (iterator.isEmpty) { + Iterator.empty + } else { + val mutableRow = new GenericMutableRow(attributes.length) + val mutablePair = new MutablePair[Any, Array[String]]() + val buffered = iterator.buffered + + // NOTE (lian): Critical path of Hive table scan, unnecessary FP style code and pattern + // matching are avoided intentionally. + val rowsAndPartitionKeys = buffered.head match { + // With partition keys + case _: Array[Any] => + buffered.map { case array: Array[Any] => + val deserializedRow = array(0) + val partitionKeys = array(1).asInstanceOf[Array[String]] + mutablePair.update(deserializedRow, partitionKeys) + } + + // Without partition keys + case _ => + val emptyPartitionKeys = Array.empty[String] + buffered.map { deserializedRow => + mutablePair.update(deserializedRow, emptyPartitionKeys) + } + } + + rowsAndPartitionKeys.map { pair => + var i = 0 + while (i < attributes.length) { + mutableRow(i) = attributeFunctions(i)(pair._1, pair._2) + i += 1 + } + mutableRow: Row + } + } + } + } + + override def output = attributes +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala new file mode 100644 index 0000000000000..594a803806ede --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -0,0 +1,248 @@ +/* + * 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.execution + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} +import org.apache.hadoop.hive.metastore.MetaStoreUtils +import org.apache.hadoop.hive.ql.Context +import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.serde2.Serializer +import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption +import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveDecimalObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharObjectInspector +import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf} + +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.execution.{SparkPlan, UnaryNode} +import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation, SparkHiveHadoopWriter} + +/** + * :: DeveloperApi :: + */ +@DeveloperApi +case class InsertIntoHiveTable( + table: MetastoreRelation, + partition: Map[String, Option[String]], + child: SparkPlan, + overwrite: Boolean) + (@transient sc: HiveContext) + extends UnaryNode { + + val outputClass = newSerializer(table.tableDesc).getSerializedClass + @transient private val hiveContext = new Context(sc.hiveconf) + @transient private val db = Hive.get(sc.hiveconf) + + private def newSerializer(tableDesc: TableDesc): Serializer = { + val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] + serializer.initialize(null, tableDesc.getProperties) + serializer + } + + override def otherCopyArgs = sc :: Nil + + def output = child.output + + /** + * Wraps with Hive types based on object inspector. + * TODO: Consolidate all hive OI/data interface code. + */ + protected def wrap(a: (Any, ObjectInspector)): Any = a match { + case (s: String, oi: JavaHiveVarcharObjectInspector) => + new HiveVarchar(s, s.size) + + case (bd: BigDecimal, oi: JavaHiveDecimalObjectInspector) => + new HiveDecimal(bd.underlying()) + + case (row: Row, oi: StandardStructObjectInspector) => + val struct = oi.create() + row.zip(oi.getAllStructFieldRefs: Seq[StructField]).foreach { + case (data, field) => + oi.setStructFieldData(struct, field, wrap(data, field.getFieldObjectInspector)) + } + struct + + case (s: Seq[_], oi: ListObjectInspector) => + val wrappedSeq = s.map(wrap(_, oi.getListElementObjectInspector)) + seqAsJavaList(wrappedSeq) + + case (obj, _) => + obj + } + + def saveAsHiveFile( + rdd: RDD[Writable], + valueClass: Class[_], + fileSinkConf: FileSinkDesc, + conf: JobConf, + isCompressed: Boolean) { + if (valueClass == null) { + throw new SparkException("Output value class not set") + } + conf.setOutputValueClass(valueClass) + if (fileSinkConf.getTableInfo.getOutputFileFormatClassName == null) { + throw new SparkException("Output format class not set") + } + // Doesn't work in Scala 2.9 due to what may be a generics bug + // TODO: Should we uncomment this for Scala 2.10? + // conf.setOutputFormat(outputFormatClass) + conf.set("mapred.output.format.class", fileSinkConf.getTableInfo.getOutputFileFormatClassName) + if (isCompressed) { + // Please note that isCompressed, "mapred.output.compress", "mapred.output.compression.codec", + // and "mapred.output.compression.type" have no impact on ORC because it uses table properties + // to store compression information. + conf.set("mapred.output.compress", "true") + fileSinkConf.setCompressed(true) + fileSinkConf.setCompressCodec(conf.get("mapred.output.compression.codec")) + fileSinkConf.setCompressType(conf.get("mapred.output.compression.type")) + } + conf.setOutputCommitter(classOf[FileOutputCommitter]) + FileOutputFormat.setOutputPath( + conf, + SparkHiveHadoopWriter.createPathFromString(fileSinkConf.getDirName, conf)) + + logger.debug("Saving as hadoop file of type " + valueClass.getSimpleName) + + val writer = new SparkHiveHadoopWriter(conf, fileSinkConf) + writer.preSetup() + + def writeToFile(context: TaskContext, iter: Iterator[Writable]) { + // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it + // around by taking a mod. We expect that no task will be attempted 2 billion times. + val attemptNumber = (context.attemptId % Int.MaxValue).toInt + + writer.setup(context.stageId, context.partitionId, attemptNumber) + writer.open() + + var count = 0 + while(iter.hasNext) { + val record = iter.next() + count += 1 + writer.write(record) + } + + writer.close() + writer.commit() + } + + sc.sparkContext.runJob(rdd, writeToFile _) + writer.commitJob() + } + + override def execute() = result + + /** + * Inserts all the rows in the table into Hive. Row objects are properly serialized with the + * `org.apache.hadoop.hive.serde2.SerDe` and the + * `org.apache.hadoop.mapred.OutputFormat` provided by the table definition. + * + * Note: this is run once and then kept to avoid double insertions. + */ + private lazy val result: RDD[Row] = { + val childRdd = child.execute() + assert(childRdd != null) + + // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer + // instances within the closure, since Serializer is not serializable while TableDesc is. + val tableDesc = table.tableDesc + val tableLocation = table.hiveQlTable.getDataLocation + val tmpLocation = hiveContext.getExternalTmpFileURI(tableLocation) + val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) + val rdd = childRdd.mapPartitions { iter => + val serializer = newSerializer(fileSinkConf.getTableInfo) + val standardOI = ObjectInspectorUtils + .getStandardObjectInspector( + fileSinkConf.getTableInfo.getDeserializer.getObjectInspector, + ObjectInspectorCopyOption.JAVA) + .asInstanceOf[StructObjectInspector] + + + val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector).toArray + val outputData = new Array[Any](fieldOIs.length) + iter.map { row => + var i = 0 + while (i < row.length) { + // Casts Strings to HiveVarchars when necessary. + outputData(i) = wrap(row(i), fieldOIs(i)) + i += 1 + } + + serializer.serialize(outputData, standardOI) + } + } + + // ORC stores compression information in table properties. While, there are other formats + // (e.g. RCFile) that rely on hadoop configurations to store compression information. + val jobConf = new JobConf(sc.hiveconf) + saveAsHiveFile( + rdd, + outputClass, + fileSinkConf, + jobConf, + sc.hiveconf.getBoolean("hive.exec.compress.output", false)) + + // TODO: Handle dynamic partitioning. + val outputPath = FileOutputFormat.getOutputPath(jobConf) + // Have to construct the format of dbname.tablename. + val qualifiedTableName = s"${table.databaseName}.${table.tableName}" + // TODO: Correctly set holdDDLTime. + // In most of the time, we should have holdDDLTime = false. + // holdDDLTime will be true when TOK_HOLD_DDLTIME presents in the query as a hint. + val holdDDLTime = false + if (partition.nonEmpty) { + val partitionSpec = partition.map { + case (key, Some(value)) => key -> value + case (key, None) => key -> "" // Should not reach here right now. + } + val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols, partitionSpec) + db.validatePartitionNameCharacters(partVals) + // inheritTableSpecs is set to true. It should be set to false for a IMPORT query + // which is currently considered as a Hive native command. + val inheritTableSpecs = true + // TODO: Correctly set isSkewedStoreAsSubdir. + val isSkewedStoreAsSubdir = false + db.loadPartition( + outputPath, + qualifiedTableName, + partitionSpec, + overwrite, + holdDDLTime, + inheritTableSpecs, + isSkewedStoreAsSubdir) + } else { + db.loadTable( + outputPath, + qualifiedTableName, + overwrite, + holdDDLTime) + } + + // It would be nice to just return the childRdd unchanged so insert operations could be chained, + // however for now we return an empty list to simplify compatibility checks with hive, which + // does not return anything for insert operations. + // TODO: implement hive compatibility as rules. + sc.sparkContext.makeRDD(Nil, 1) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala new file mode 100644 index 0000000000000..fe6031678f70f --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow, Row} +import org.apache.spark.sql.execution.{Command, LeafNode} +import org.apache.spark.sql.hive.HiveContext + +/** + * :: DeveloperApi :: + */ +@DeveloperApi +case class NativeCommand( + sql: String, output: Seq[Attribute])( + @transient context: HiveContext) + extends LeafNode with Command { + + override protected[sql] lazy val sideEffectResult: Seq[String] = context.runSqlHive(sql) + + override def execute(): RDD[Row] = { + if (sideEffectResult.size == 0) { + context.emptyResult + } else { + val rows = sideEffectResult.map(r => new GenericRow(Array[Any](r))) + context.sparkContext.parallelize(rows, 1) + } + } + + override def otherCopyArgs = context :: Nil +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/hiveOperators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/hiveOperators.scala deleted file mode 100644 index 2de2db28a7e04..0000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/hiveOperators.scala +++ /dev/null @@ -1,524 +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.execution - -import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.MetaStoreUtils -import org.apache.hadoop.hive.metastore.api.FieldSchema -import org.apache.hadoop.hive.ql.Context -import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Hive} -import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatUtils -import org.apache.hadoop.hive.ql.plan.{TableDesc, FileSinkDesc} -import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption -import org.apache.hadoop.hive.serde2.objectinspector._ -import org.apache.hadoop.hive.serde2.objectinspector.primitive._ -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils -import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Serializer} -import org.apache.hadoop.io.Writable -import org.apache.hadoop.mapred._ - -import org.apache.spark -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types.{BooleanType, DataType} -import org.apache.spark.sql.execution._ -import org.apache.spark.sql.hive._ -import org.apache.spark.util.MutablePair -import org.apache.spark.{TaskContext, SparkException} - -/* Implicits */ -import scala.collection.JavaConversions._ - -/** - * :: DeveloperApi :: - * The Hive table scan operator. Column and partition pruning are both handled. - * - * @param attributes Attributes to be fetched from the Hive table. - * @param relation The Hive table be be scanned. - * @param partitionPruningPred An optional partition pruning predicate for partitioned table. - */ -@DeveloperApi -case class HiveTableScan( - attributes: Seq[Attribute], - relation: MetastoreRelation, - partitionPruningPred: Option[Expression])( - @transient val context: HiveContext) - extends LeafNode - with HiveInspectors { - - require(partitionPruningPred.isEmpty || relation.hiveQlTable.isPartitioned, - "Partition pruning predicates only supported for partitioned tables.") - - // Bind all partition key attribute references in the partition pruning predicate for later - // evaluation. - private val boundPruningPred = partitionPruningPred.map { pred => - require( - pred.dataType == BooleanType, - s"Data type of predicate $pred must be BooleanType rather than ${pred.dataType}.") - - BindReferences.bindReference(pred, relation.partitionKeys) - } - - @transient - val hadoopReader = new HadoopTableReader(relation.tableDesc, context) - - /** - * The hive object inspector for this table, which can be used to extract values from the - * serialized row representation. - */ - @transient - lazy val objectInspector = - relation.tableDesc.getDeserializer.getObjectInspector.asInstanceOf[StructObjectInspector] - - /** - * Functions that extract the requested attributes from the hive output. Partitioned values are - * casted from string to its declared data type. - */ - @transient - protected lazy val attributeFunctions: Seq[(Any, Array[String]) => Any] = { - attributes.map { a => - val ordinal = relation.partitionKeys.indexOf(a) - if (ordinal >= 0) { - val dataType = relation.partitionKeys(ordinal).dataType - (_: Any, partitionKeys: Array[String]) => { - castFromString(partitionKeys(ordinal), dataType) - } - } else { - val ref = objectInspector.getAllStructFieldRefs - .find(_.getFieldName == a.name) - .getOrElse(sys.error(s"Can't find attribute $a")) - val fieldObjectInspector = ref.getFieldObjectInspector - - val unwrapHiveData = fieldObjectInspector match { - case _: HiveVarcharObjectInspector => - (value: Any) => value.asInstanceOf[HiveVarchar].getValue - case _: HiveDecimalObjectInspector => - (value: Any) => BigDecimal(value.asInstanceOf[HiveDecimal].bigDecimalValue()) - case _ => - identity[Any] _ - } - - (row: Any, _: Array[String]) => { - val data = objectInspector.getStructFieldData(row, ref) - val hiveData = unwrapData(data, fieldObjectInspector) - if (hiveData != null) unwrapHiveData(hiveData) else null - } - } - } - } - - private def castFromString(value: String, dataType: DataType) = { - Cast(Literal(value), dataType).eval(null) - } - - private def addColumnMetadataToConf(hiveConf: HiveConf) { - // Specifies IDs and internal names of columns to be scanned. - val neededColumnIDs = attributes.map(a => relation.output.indexWhere(_.name == a.name): Integer) - val columnInternalNames = neededColumnIDs.map(HiveConf.getColumnInternalName(_)).mkString(",") - - if (attributes.size == relation.output.size) { - ColumnProjectionUtils.setFullyReadColumns(hiveConf) - } else { - ColumnProjectionUtils.appendReadColumnIDs(hiveConf, neededColumnIDs) - } - - ColumnProjectionUtils.appendReadColumnNames(hiveConf, attributes.map(_.name)) - - // Specifies types and object inspectors of columns to be scanned. - val structOI = ObjectInspectorUtils - .getStandardObjectInspector( - relation.tableDesc.getDeserializer.getObjectInspector, - ObjectInspectorCopyOption.JAVA) - .asInstanceOf[StructObjectInspector] - - val columnTypeNames = structOI - .getAllStructFieldRefs - .map(_.getFieldObjectInspector) - .map(TypeInfoUtils.getTypeInfoFromObjectInspector(_).getTypeName) - .mkString(",") - - hiveConf.set(serdeConstants.LIST_COLUMN_TYPES, columnTypeNames) - hiveConf.set(serdeConstants.LIST_COLUMNS, columnInternalNames) - } - - addColumnMetadataToConf(context.hiveconf) - - @transient - def inputRdd = if (!relation.hiveQlTable.isPartitioned) { - hadoopReader.makeRDDForTable(relation.hiveQlTable) - } else { - hadoopReader.makeRDDForPartitionedTable(prunePartitions(relation.hiveQlPartitions)) - } - - /** - * Prunes partitions not involve the query plan. - * - * @param partitions All partitions of the relation. - * @return Partitions that are involved in the query plan. - */ - private[hive] def prunePartitions(partitions: Seq[HivePartition]) = { - boundPruningPred match { - case None => partitions - case Some(shouldKeep) => partitions.filter { part => - val dataTypes = relation.partitionKeys.map(_.dataType) - val castedValues = for ((value, dataType) <- part.getValues.zip(dataTypes)) yield { - castFromString(value, dataType) - } - - // Only partitioned values are needed here, since the predicate has already been bound to - // partition key attribute references. - val row = new GenericRow(castedValues.toArray) - shouldKeep.eval(row).asInstanceOf[Boolean] - } - } - } - - def execute() = { - inputRdd.mapPartitions { iterator => - if (iterator.isEmpty) { - Iterator.empty - } else { - val mutableRow = new GenericMutableRow(attributes.length) - val mutablePair = new MutablePair[Any, Array[String]]() - val buffered = iterator.buffered - - // NOTE (lian): Critical path of Hive table scan, unnecessary FP style code and pattern - // matching are avoided intentionally. - val rowsAndPartitionKeys = buffered.head match { - // With partition keys - case _: Array[Any] => - buffered.map { case array: Array[Any] => - val deserializedRow = array(0) - val partitionKeys = array(1).asInstanceOf[Array[String]] - mutablePair.update(deserializedRow, partitionKeys) - } - - // Without partition keys - case _ => - val emptyPartitionKeys = Array.empty[String] - buffered.map { deserializedRow => - mutablePair.update(deserializedRow, emptyPartitionKeys) - } - } - - rowsAndPartitionKeys.map { pair => - var i = 0 - while (i < attributes.length) { - mutableRow(i) = attributeFunctions(i)(pair._1, pair._2) - i += 1 - } - mutableRow: Row - } - } - } - } - - def output = attributes -} - -/** - * :: DeveloperApi :: - */ -@DeveloperApi -case class InsertIntoHiveTable( - table: MetastoreRelation, - partition: Map[String, Option[String]], - child: SparkPlan, - overwrite: Boolean) - (@transient sc: HiveContext) - extends UnaryNode { - - val outputClass = newSerializer(table.tableDesc).getSerializedClass - @transient private val hiveContext = new Context(sc.hiveconf) - @transient private val db = Hive.get(sc.hiveconf) - - private def newSerializer(tableDesc: TableDesc): Serializer = { - val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] - serializer.initialize(null, tableDesc.getProperties) - serializer - } - - override def otherCopyArgs = sc :: Nil - - def output = child.output - - /** - * Wraps with Hive types based on object inspector. - * TODO: Consolidate all hive OI/data interface code. - */ - protected def wrap(a: (Any, ObjectInspector)): Any = a match { - case (s: String, oi: JavaHiveVarcharObjectInspector) => - new HiveVarchar(s, s.size) - - case (bd: BigDecimal, oi: JavaHiveDecimalObjectInspector) => - new HiveDecimal(bd.underlying()) - - case (row: Row, oi: StandardStructObjectInspector) => - val struct = oi.create() - row.zip(oi.getAllStructFieldRefs: Seq[StructField]).foreach { - case (data, field) => - oi.setStructFieldData(struct, field, wrap(data, field.getFieldObjectInspector)) - } - struct - - case (s: Seq[_], oi: ListObjectInspector) => - val wrappedSeq = s.map(wrap(_, oi.getListElementObjectInspector)) - seqAsJavaList(wrappedSeq) - - case (obj, _) => - obj - } - - def saveAsHiveFile( - rdd: RDD[Writable], - valueClass: Class[_], - fileSinkConf: FileSinkDesc, - conf: JobConf, - isCompressed: Boolean) { - if (valueClass == null) { - throw new SparkException("Output value class not set") - } - conf.setOutputValueClass(valueClass) - if (fileSinkConf.getTableInfo.getOutputFileFormatClassName == null) { - throw new SparkException("Output format class not set") - } - // Doesn't work in Scala 2.9 due to what may be a generics bug - // TODO: Should we uncomment this for Scala 2.10? - // conf.setOutputFormat(outputFormatClass) - conf.set("mapred.output.format.class", fileSinkConf.getTableInfo.getOutputFileFormatClassName) - if (isCompressed) { - // Please note that isCompressed, "mapred.output.compress", "mapred.output.compression.codec", - // and "mapred.output.compression.type" have no impact on ORC because it uses table properties - // to store compression information. - conf.set("mapred.output.compress", "true") - fileSinkConf.setCompressed(true) - fileSinkConf.setCompressCodec(conf.get("mapred.output.compression.codec")) - fileSinkConf.setCompressType(conf.get("mapred.output.compression.type")) - } - conf.setOutputCommitter(classOf[FileOutputCommitter]) - FileOutputFormat.setOutputPath( - conf, - SparkHiveHadoopWriter.createPathFromString(fileSinkConf.getDirName, conf)) - - logger.debug("Saving as hadoop file of type " + valueClass.getSimpleName) - - val writer = new SparkHiveHadoopWriter(conf, fileSinkConf) - writer.preSetup() - - def writeToFile(context: TaskContext, iter: Iterator[Writable]) { - // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it - // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.attemptId % Int.MaxValue).toInt - - writer.setup(context.stageId, context.partitionId, attemptNumber) - writer.open() - - var count = 0 - while(iter.hasNext) { - val record = iter.next() - count += 1 - writer.write(record) - } - - writer.close() - writer.commit() - } - - sc.sparkContext.runJob(rdd, writeToFile _) - writer.commitJob() - } - - override def execute() = result - - /** - * Inserts all the rows in the table into Hive. Row objects are properly serialized with the - * `org.apache.hadoop.hive.serde2.SerDe` and the - * `org.apache.hadoop.mapred.OutputFormat` provided by the table definition. - * - * Note: this is run once and then kept to avoid double insertions. - */ - private lazy val result: RDD[Row] = { - val childRdd = child.execute() - assert(childRdd != null) - - // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer - // instances within the closure, since Serializer is not serializable while TableDesc is. - val tableDesc = table.tableDesc - val tableLocation = table.hiveQlTable.getDataLocation - val tmpLocation = hiveContext.getExternalTmpFileURI(tableLocation) - val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) - val rdd = childRdd.mapPartitions { iter => - val serializer = newSerializer(fileSinkConf.getTableInfo) - val standardOI = ObjectInspectorUtils - .getStandardObjectInspector( - fileSinkConf.getTableInfo.getDeserializer.getObjectInspector, - ObjectInspectorCopyOption.JAVA) - .asInstanceOf[StructObjectInspector] - - - val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector).toArray - val outputData = new Array[Any](fieldOIs.length) - iter.map { row => - var i = 0 - while (i < row.length) { - // Casts Strings to HiveVarchars when necessary. - outputData(i) = wrap(row(i), fieldOIs(i)) - i += 1 - } - - serializer.serialize(outputData, standardOI) - } - } - - // ORC stores compression information in table properties. While, there are other formats - // (e.g. RCFile) that rely on hadoop configurations to store compression information. - val jobConf = new JobConf(sc.hiveconf) - saveAsHiveFile( - rdd, - outputClass, - fileSinkConf, - jobConf, - sc.hiveconf.getBoolean("hive.exec.compress.output", false)) - - // TODO: Handle dynamic partitioning. - val outputPath = FileOutputFormat.getOutputPath(jobConf) - // Have to construct the format of dbname.tablename. - val qualifiedTableName = s"${table.databaseName}.${table.tableName}" - // TODO: Correctly set holdDDLTime. - // In most of the time, we should have holdDDLTime = false. - // holdDDLTime will be true when TOK_HOLD_DDLTIME presents in the query as a hint. - val holdDDLTime = false - if (partition.nonEmpty) { - val partitionSpec = partition.map { - case (key, Some(value)) => key -> value - case (key, None) => key -> "" // Should not reach here right now. - } - val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols, partitionSpec) - db.validatePartitionNameCharacters(partVals) - // inheritTableSpecs is set to true. It should be set to false for a IMPORT query - // which is currently considered as a Hive native command. - val inheritTableSpecs = true - // TODO: Correctly set isSkewedStoreAsSubdir. - val isSkewedStoreAsSubdir = false - db.loadPartition( - outputPath, - qualifiedTableName, - partitionSpec, - overwrite, - holdDDLTime, - inheritTableSpecs, - isSkewedStoreAsSubdir) - } else { - db.loadTable( - outputPath, - qualifiedTableName, - overwrite, - holdDDLTime) - } - - // It would be nice to just return the childRdd unchanged so insert operations could be chained, - // however for now we return an empty list to simplify compatibility checks with hive, which - // does not return anything for insert operations. - // TODO: implement hive compatibility as rules. - sc.sparkContext.makeRDD(Nil, 1) - } -} - -/** - * :: DeveloperApi :: - */ -@DeveloperApi -case class NativeCommand( - sql: String, output: Seq[Attribute])( - @transient context: HiveContext) - extends LeafNode with Command { - - override protected[sql] lazy val sideEffectResult: Seq[String] = context.runSqlHive(sql) - - override def execute(): RDD[spark.sql.Row] = { - if (sideEffectResult.size == 0) { - context.emptyResult - } else { - val rows = sideEffectResult.map(r => new GenericRow(Array[Any](r))) - context.sparkContext.parallelize(rows, 1) - } - } - - override def otherCopyArgs = context :: Nil -} - -/** - * :: DeveloperApi :: - */ -@DeveloperApi -case class DescribeHiveTableCommand( - table: MetastoreRelation, - output: Seq[Attribute], - isExtended: Boolean)( - @transient context: HiveContext) - extends LeafNode with Command { - - // Strings with the format like Hive. It is used for result comparison in our unit tests. - lazy val hiveString: Seq[String] = { - val alignment = 20 - val delim = "\t" - - sideEffectResult.map { - case (name, dataType, comment) => - String.format("%-" + alignment + "s", name) + delim + - String.format("%-" + alignment + "s", dataType) + delim + - String.format("%-" + alignment + "s", Option(comment).getOrElse("None")) - } - } - - override protected[sql] lazy val sideEffectResult: Seq[(String, String, String)] = { - // Trying to mimic the format of Hive's output. But not exactly the same. - var results: Seq[(String, String, String)] = Nil - - val columns: Seq[FieldSchema] = table.hiveQlTable.getCols - val partitionColumns: Seq[FieldSchema] = table.hiveQlTable.getPartCols - results ++= columns.map(field => (field.getName, field.getType, field.getComment)) - if (!partitionColumns.isEmpty) { - val partColumnInfo = - partitionColumns.map(field => (field.getName, field.getType, field.getComment)) - results ++= - partColumnInfo ++ - Seq(("# Partition Information", "", "")) ++ - Seq((s"# ${output.get(0).name}", output.get(1).name, output.get(2).name)) ++ - partColumnInfo - } - - if (isExtended) { - results ++= Seq(("Detailed Table Information", table.hiveQlTable.getTTable.toString, "")) - } - - results - } - - override def execute(): RDD[Row] = { - val rows = sideEffectResult.map { - case (name, dataType, comment) => new GenericRow(Array[Any](name, dataType, comment)) - } - context.sparkContext.parallelize(rows, 1) - } - - override def otherCopyArgs = context :: Nil -} From 0a432d6a05d2a8f1f8f5e6e851f5174c98a64d1d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 21 Jun 2014 13:02:49 -0700 Subject: [PATCH 012/124] HOTFIX: Fix missing MIMA ignore --- dev/mima | 1 + project/MimaExcludes.scala | 2 ++ 2 files changed, 3 insertions(+) diff --git a/dev/mima b/dev/mima index b68800d6d0173..7857294f61caf 100755 --- a/dev/mima +++ b/dev/mima @@ -18,6 +18,7 @@ # set -o pipefail +set -e # Go to the Spark project root directory FWDIR="$(cd `dirname $0`/..; pwd)" diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 042fdfcc47261..af620d61607fa 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,6 +36,8 @@ object MimaExcludes { case v if v.startsWith("1.1") => Seq(MimaBuild.excludeSparkPackage("graphx")) ++ Seq( + // Adding new method to JavaRDLike trait - we should probably mark this as a developer API. + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitions"), // We made a mistake earlier (ed06500d3) in the Java API to use default parameter values // for countApproxDistinct* functions, which does not work in Java. We later removed // them, and use the following to tell Mima to not care about them. From 3e0b078001ae9322294b357022fed230cecb38bd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 21 Jun 2014 15:20:15 -0700 Subject: [PATCH 013/124] HOTFIX: Add excludes for new MIMA files --- .rat-excludes | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.rat-excludes b/.rat-excludes index 15344dfb292db..796c32a80896c 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -4,6 +4,8 @@ target .classpath .mima-excludes .generated-mima-excludes +.generated-mima-class-excludes +.generated-mima-member-excludes .rat-excludes .*md derby.log From 1db9cbc33603cfe012790b7dd4975df20ab15bda Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 21 Jun 2014 23:29:57 -0700 Subject: [PATCH 014/124] SPARK-1996. Remove use of special Maven repo for Akka Just following up Matei's suggestion to remove the Akka repo references. Builds and the audit-release script appear OK. Author: Sean Owen Closes #1170 from srowen/SPARK-1996 and squashes the following commits: 5ca2930 [Sean Owen] Remove outdated Akka repository references --- dev/audit-release/blank_maven_build/pom.xml | 4 ---- dev/audit-release/blank_sbt_build/build.sbt | 1 - dev/audit-release/maven_app_core/pom.xml | 4 ---- dev/audit-release/sbt_app_core/build.sbt | 1 - dev/audit-release/sbt_app_ganglia/build.sbt | 1 - dev/audit-release/sbt_app_graphx/build.sbt | 1 - dev/audit-release/sbt_app_hive/build.sbt | 1 - dev/audit-release/sbt_app_sql/build.sbt | 1 - dev/audit-release/sbt_app_streaming/build.sbt | 1 - docs/quick-start.md | 8 -------- 10 files changed, 23 deletions(-) diff --git a/dev/audit-release/blank_maven_build/pom.xml b/dev/audit-release/blank_maven_build/pom.xml index 047659e4a8b7c..02dd9046c9a49 100644 --- a/dev/audit-release/blank_maven_build/pom.xml +++ b/dev/audit-release/blank_maven_build/pom.xml @@ -28,10 +28,6 @@ Spray.cc repository http://repo.spray.cc - - Akka repository - http://repo.akka.io/releases - Spark Staging Repo ${spark.release.repository} diff --git a/dev/audit-release/blank_sbt_build/build.sbt b/dev/audit-release/blank_sbt_build/build.sbt index 1cf52743f27f4..696c7f651837c 100644 --- a/dev/audit-release/blank_sbt_build/build.sbt +++ b/dev/audit-release/blank_sbt_build/build.sbt @@ -25,5 +25,4 @@ libraryDependencies += "org.apache.spark" % System.getenv.get("SPARK_MODULE") % resolvers ++= Seq( "Spark Release Repository" at System.getenv.get("SPARK_RELEASE_REPOSITORY"), - "Akka Repository" at "http://repo.akka.io/releases/", "Spray Repository" at "http://repo.spray.cc/") diff --git a/dev/audit-release/maven_app_core/pom.xml b/dev/audit-release/maven_app_core/pom.xml index 76a381f8e17e0..b516396825573 100644 --- a/dev/audit-release/maven_app_core/pom.xml +++ b/dev/audit-release/maven_app_core/pom.xml @@ -28,10 +28,6 @@ Spray.cc repository http://repo.spray.cc - - Akka repository - http://repo.akka.io/releases - Spark Staging Repo ${spark.release.repository} diff --git a/dev/audit-release/sbt_app_core/build.sbt b/dev/audit-release/sbt_app_core/build.sbt index 97a8cc3a4e095..291b1d6440bac 100644 --- a/dev/audit-release/sbt_app_core/build.sbt +++ b/dev/audit-release/sbt_app_core/build.sbt @@ -25,5 +25,4 @@ libraryDependencies += "org.apache.spark" %% "spark-core" % System.getenv.get("S resolvers ++= Seq( "Spark Release Repository" at System.getenv.get("SPARK_RELEASE_REPOSITORY"), - "Akka Repository" at "http://repo.akka.io/releases/", "Spray Repository" at "http://repo.spray.cc/") diff --git a/dev/audit-release/sbt_app_ganglia/build.sbt b/dev/audit-release/sbt_app_ganglia/build.sbt index 55db675c722d1..6d9474acf5bbc 100644 --- a/dev/audit-release/sbt_app_ganglia/build.sbt +++ b/dev/audit-release/sbt_app_ganglia/build.sbt @@ -27,5 +27,4 @@ libraryDependencies += "org.apache.spark" %% "spark-ganglia-lgpl" % System.geten resolvers ++= Seq( "Spark Release Repository" at System.getenv.get("SPARK_RELEASE_REPOSITORY"), - "Akka Repository" at "http://repo.akka.io/releases/", "Spray Repository" at "http://repo.spray.cc/") diff --git a/dev/audit-release/sbt_app_graphx/build.sbt b/dev/audit-release/sbt_app_graphx/build.sbt index 66f2db357d49b..dd11245e67d44 100644 --- a/dev/audit-release/sbt_app_graphx/build.sbt +++ b/dev/audit-release/sbt_app_graphx/build.sbt @@ -25,5 +25,4 @@ libraryDependencies += "org.apache.spark" %% "spark-graphx" % System.getenv.get( resolvers ++= Seq( "Spark Release Repository" at System.getenv.get("SPARK_RELEASE_REPOSITORY"), - "Akka Repository" at "http://repo.akka.io/releases/", "Spray Repository" at "http://repo.spray.cc/") diff --git a/dev/audit-release/sbt_app_hive/build.sbt b/dev/audit-release/sbt_app_hive/build.sbt index 7ac1be729c561..a0d4f25da5842 100644 --- a/dev/audit-release/sbt_app_hive/build.sbt +++ b/dev/audit-release/sbt_app_hive/build.sbt @@ -25,5 +25,4 @@ libraryDependencies += "org.apache.spark" %% "spark-hive" % System.getenv.get("S resolvers ++= Seq( "Spark Release Repository" at System.getenv.get("SPARK_RELEASE_REPOSITORY"), - "Akka Repository" at "http://repo.akka.io/releases/", "Spray Repository" at "http://repo.spray.cc/") diff --git a/dev/audit-release/sbt_app_sql/build.sbt b/dev/audit-release/sbt_app_sql/build.sbt index 6e0ad3b4b2960..9116180f71a44 100644 --- a/dev/audit-release/sbt_app_sql/build.sbt +++ b/dev/audit-release/sbt_app_sql/build.sbt @@ -25,5 +25,4 @@ libraryDependencies += "org.apache.spark" %% "spark-sql" % System.getenv.get("SP resolvers ++= Seq( "Spark Release Repository" at System.getenv.get("SPARK_RELEASE_REPOSITORY"), - "Akka Repository" at "http://repo.akka.io/releases/", "Spray Repository" at "http://repo.spray.cc/") diff --git a/dev/audit-release/sbt_app_streaming/build.sbt b/dev/audit-release/sbt_app_streaming/build.sbt index 492e5e7c8d763..cb369d516dd16 100644 --- a/dev/audit-release/sbt_app_streaming/build.sbt +++ b/dev/audit-release/sbt_app_streaming/build.sbt @@ -25,5 +25,4 @@ libraryDependencies += "org.apache.spark" %% "spark-streaming" % System.getenv.g resolvers ++= Seq( "Spark Release Repository" at System.getenv.get("SPARK_RELEASE_REPOSITORY"), - "Akka Repository" at "http://repo.akka.io/releases/", "Spray Repository" at "http://repo.spray.cc/") diff --git a/docs/quick-start.md b/docs/quick-start.md index 64023994771b7..23313d8aa6152 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -266,8 +266,6 @@ version := "1.0" scalaVersion := "{{site.SCALA_VERSION}}" libraryDependencies += "org.apache.spark" %% "spark-core" % "{{site.SPARK_VERSION}}" - -resolvers += "Akka Repository" at "http://repo.akka.io/releases/" {% endhighlight %} For sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` @@ -349,12 +347,6 @@ Note that Spark artifacts are tagged with a Scala version. Simple Project jar 1.0 - - - Akka repository - http://repo.akka.io/releases - - org.apache.spark From 58b32f3470f9fa67ad7dffb7d3dcd2a954b4b4e9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 22 Jun 2014 00:55:27 -0700 Subject: [PATCH 015/124] SPARK-2231: dev/run-tests should include YARN and use a recent Hadoop version ...rsion Author: Patrick Wendell Closes #1175 from pwendell/test-hadoop-version and squashes the following commits: 9210ef4 [Patrick Wendell] SPARK-2231: dev/run-tests should include YARN and use a recent Hadoop version --- dev/run-tests | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dev/run-tests b/dev/run-tests index c82a47ebb618b..d9df020f7563c 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -21,6 +21,9 @@ FWDIR="$(cd `dirname $0`/..; pwd)" cd $FWDIR +export SPARK_HADOOP_VERSION=2.3.0 +export SPARK_YARN=true + # Remove work directory rm -rf ./work From 476581e8c8ca03a5940c404fee8a06361ff94cb5 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 22 Jun 2014 01:12:15 -0700 Subject: [PATCH 016/124] SPARK-2034. KafkaInputDStream doesn't close resources and may prevent JVM shutdown MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Tobias noted today on the mailing list: ======== I am trying to use Spark Streaming with Kafka, which works like a charm – except for shutdown. When I run my program with "sbt run-main", sbt will never exit, because there are two non-daemon threads left that don't die. I created a minimal example at . It starts a StreamingContext and does nothing more than connecting to a Kafka server and printing what it receives. Using the `future Unknown macro: { ... } ` construct, I shut down the StreamingContext after some seconds and then print the difference between the threads at start time and at end time. The output can be found at . There are a number of threads remaining that will prevent sbt from exiting. When I replace `KafkaUtils.createStream(...)` with a call that does exactly the same, except that it calls `consumerConnector.shutdown()` in `KafkaReceiver.onStop()` (which it should, IMO), the output is as shown at . Does anyone have any idea what is going on here and why the program doesn't shut down properly? The behavior is the same with both kafka 0.8.0 and 0.8.1.1, by the way. ======== Something similar was noted last year: http://mail-archives.apache.org/mod_mbox/spark-dev/201309.mbox/%3C1380220041.2428.YahooMailNeo@web160804.mail.bf1.yahoo.com%3E KafkaInputDStream doesn't close `ConsumerConnector` in `onStop()`, and does not close the `Executor` it creates. The latter leaves non-daemon threads and can prevent the JVM from shutting down even if streaming is closed properly. Author: Sean Owen Closes #980 from srowen/SPARK-2034 and squashes the following commits: 9f31a8d [Sean Owen] Restore ClassTag to private class because MIMA flags it; is the shadowing intended? 2d579a8 [Sean Owen] Close ConsumerConnector in onStop; shutdown() the local Executor that is created so that its threads stop when done; close the Zookeeper client even on exception; fix a few typos; log exceptions that otherwise vanish --- .../streaming/kafka/KafkaInputDStream.scala | 55 +++++++++++-------- 1 file changed, 33 insertions(+), 22 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index 21443ebbbfb0e..38095e88dcea9 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -38,7 +38,7 @@ import org.apache.spark.streaming.receiver.Receiver /** * Input stream that pulls messages from a Kafka Broker. * - * @param kafkaParams Map of kafka configuration paramaters. + * @param kafkaParams Map of kafka configuration parameters. * See: http://kafka.apache.org/configuration.html * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. @@ -76,29 +76,31 @@ class KafkaReceiver[ // Connection to Kafka var consumerConnector : ConsumerConnector = null - def onStop() { } + def onStop() { + if (consumerConnector != null) { + consumerConnector.shutdown() + } + } def onStart() { - // In case we are using multiple Threads to handle Kafka Messages - val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _)) - logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("group.id")) // Kafka connection properties val props = new Properties() kafkaParams.foreach(param => props.put(param._1, param._2)) + val zkConnect = kafkaParams("zookeeper.connect") // Create the connection to the cluster - logInfo("Connecting to Zookeper: " + kafkaParams("zookeeper.connect")) + logInfo("Connecting to Zookeeper: " + zkConnect) val consumerConfig = new ConsumerConfig(props) consumerConnector = Consumer.create(consumerConfig) - logInfo("Connected to " + kafkaParams("zookeeper.connect")) + logInfo("Connected to " + zkConnect) - // When autooffset.reset is defined, it is our responsibility to try and whack the + // When auto.offset.reset is defined, it is our responsibility to try and whack the // consumer group zk node. if (kafkaParams.contains("auto.offset.reset")) { - tryZookeeperConsumerGroupCleanup(kafkaParams("zookeeper.connect"), kafkaParams("group.id")) + tryZookeeperConsumerGroupCleanup(zkConnect, kafkaParams("group.id")) } val keyDecoder = manifest[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) @@ -112,10 +114,14 @@ class KafkaReceiver[ val topicMessageStreams = consumerConnector.createMessageStreams( topics, keyDecoder, valueDecoder) - - // Start the messages handler for each partition - topicMessageStreams.values.foreach { streams => - streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) } + val executorPool = Executors.newFixedThreadPool(topics.values.sum) + try { + // Start the messages handler for each partition + topicMessageStreams.values.foreach { streams => + streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) } + } + } finally { + executorPool.shutdown() // Just causes threads to terminate after work is done } } @@ -124,30 +130,35 @@ class KafkaReceiver[ extends Runnable { def run() { logInfo("Starting MessageHandler.") - for (msgAndMetadata <- stream) { - store((msgAndMetadata.key, msgAndMetadata.message)) + try { + for (msgAndMetadata <- stream) { + store((msgAndMetadata.key, msgAndMetadata.message)) + } + } catch { + case e: Throwable => logError("Error handling message; exiting", e) } } } - // It is our responsibility to delete the consumer group when specifying autooffset.reset. This + // It is our responsibility to delete the consumer group when specifying auto.offset.reset. This // is because Kafka 0.7.2 only honors this param when the group is not in zookeeper. // // The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied - // from Kafkas' ConsoleConsumer. See code related to 'autooffset.reset' when it is set to + // from Kafka's ConsoleConsumer. See code related to 'auto.offset.reset' when it is set to // 'smallest'/'largest': // scalastyle:off // https://github.com/apache/kafka/blob/0.7.2/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala // scalastyle:on private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) { + val dir = "/consumers/" + groupId + logInfo("Cleaning up temporary Zookeeper data under " + dir + ".") + val zk = new ZkClient(zkUrl, 30*1000, 30*1000, ZKStringSerializer) try { - val dir = "/consumers/" + groupId - logInfo("Cleaning up temporary zookeeper data under " + dir + ".") - val zk = new ZkClient(zkUrl, 30*1000, 30*1000, ZKStringSerializer) zk.deleteRecursive(dir) - zk.close() } catch { - case _ : Throwable => // swallow + case e: Throwable => logWarning("Error cleaning up temporary Zookeeper data", e) + } finally { + zk.close() } } } From 9fe28c35df0dc8ac4a54db2a528fb7ae56d3f978 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 22 Jun 2014 11:47:49 -0700 Subject: [PATCH 017/124] SPARK-1316. Remove use of Commons IO Commons IO is actually barely used, and is not a declared dependency. This just replaces with equivalents from the JDK and Guava. Author: Sean Owen Closes #1173 from srowen/SPARK-1316 and squashes the following commits: 2eb53db [Sean Owen] Reorder Guava import 8fde404 [Sean Owen] Remove use of Commons IO, which is not actually a dependency --- .../util/logging/RollingFileAppender.scala | 6 +++--- .../apache/spark/util/FileAppenderSuite.scala | 17 ++++++++++------- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala index 1bbbd20cf076f..e579421676343 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala @@ -19,7 +19,7 @@ package org.apache.spark.util.logging import java.io.{File, FileFilter, InputStream} -import org.apache.commons.io.FileUtils +import com.google.common.io.Files import org.apache.spark.SparkConf import RollingFileAppender._ @@ -83,7 +83,7 @@ private[spark] class RollingFileAppender( logDebug(s"Attempting to rollover file $activeFile to file $rolloverFile") if (activeFile.exists) { if (!rolloverFile.exists) { - FileUtils.moveFile(activeFile, rolloverFile) + Files.move(activeFile, rolloverFile) logInfo(s"Rolled over $activeFile to $rolloverFile") } else { // In case the rollover file name clashes, make a unique file name. @@ -100,7 +100,7 @@ private[spark] class RollingFileAppender( logWarning(s"Rollover file $rolloverFile already exists, " + s"rolled over $activeFile to file $altRolloverFile") - FileUtils.moveFile(activeFile, altRolloverFile) + Files.move(activeFile, altRolloverFile) } } else { logWarning(s"File $activeFile does not exist") diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index 02e228945bbd9..ca37d707b06ca 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -18,13 +18,16 @@ package org.apache.spark.util import java.io._ +import java.nio.charset.Charset import scala.collection.mutable.HashSet import scala.reflect._ -import org.apache.commons.io.{FileUtils, IOUtils} -import org.apache.spark.{Logging, SparkConf} import org.scalatest.{BeforeAndAfter, FunSuite} + +import com.google.common.io.Files + +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.util.logging.{RollingFileAppender, SizeBasedRollingPolicy, TimeBasedRollingPolicy, FileAppender} class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { @@ -41,11 +44,11 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { test("basic file appender") { val testString = (1 to 1000).mkString(", ") - val inputStream = IOUtils.toInputStream(testString) + val inputStream = new ByteArrayInputStream(testString.getBytes(Charset.forName("UTF-8"))) val appender = new FileAppender(inputStream, testFile) inputStream.close() appender.awaitTermination() - assert(FileUtils.readFileToString(testFile) === testString) + assert(Files.toString(testFile, Charset.forName("UTF-8")) === testString) } test("rolling file appender - time-based rolling") { @@ -93,7 +96,7 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { val allGeneratedFiles = new HashSet[String]() val items = (1 to 10).map { _.toString * 10000 } for (i <- 0 until items.size) { - testOutputStream.write(items(i).getBytes("UTF8")) + testOutputStream.write(items(i).getBytes(Charset.forName("UTF-8"))) testOutputStream.flush() allGeneratedFiles ++= RollingFileAppender.getSortedRolledOverFiles( testFile.getParentFile.toString, testFile.getName).map(_.toString) @@ -197,7 +200,7 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { // send data to appender through the input stream, and wait for the data to be written val expectedText = textToAppend.mkString("") for (i <- 0 until textToAppend.size) { - outputStream.write(textToAppend(i).getBytes("UTF8")) + outputStream.write(textToAppend(i).getBytes(Charset.forName("UTF-8"))) outputStream.flush() Thread.sleep(sleepTimeBetweenTexts) } @@ -212,7 +215,7 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { logInfo("Filtered files: \n" + generatedFiles.mkString("\n")) assert(generatedFiles.size > 1) val allText = generatedFiles.map { file => - FileUtils.readFileToString(file) + Files.toString(file, Charset.forName("UTF-8")) }.mkString("") assert(allText === expectedText) generatedFiles From 409d24e2b2e52df8fd2c5aca69def29b9a097a79 Mon Sep 17 00:00:00 2001 From: witgo Date: Sun, 22 Jun 2014 18:25:16 -0700 Subject: [PATCH 018/124] SPARK-2229: FileAppender throw an llegalArgumentException in jdk6 Author: witgo Closes #1174 from witgo/SPARK-2229 and squashes the following commits: f85f321 [witgo] FileAppender throw anIllegalArgumentException in JDK6 e1a8da8 [witgo] SizeBasedRollingPolicy throw an java.lang.IllegalArgumentException in JDK6 --- .../org/apache/spark/util/logging/FileAppender.scala | 8 ++++---- .../org/apache/spark/util/logging/RollingPolicy.scala | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala index 8e9c3036d09c2..1d5467060623c 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala @@ -125,16 +125,16 @@ private[spark] object FileAppender extends Logging { val validatedParams: Option[(Long, String)] = rollingInterval match { case "daily" => logInfo(s"Rolling executor logs enabled for $file with daily rolling") - Some(24 * 60 * 60 * 1000L, "--YYYY-MM-dd") + Some(24 * 60 * 60 * 1000L, "--yyyy-MM-dd") case "hourly" => logInfo(s"Rolling executor logs enabled for $file with hourly rolling") - Some(60 * 60 * 1000L, "--YYYY-MM-dd--HH") + Some(60 * 60 * 1000L, "--yyyy-MM-dd--HH") case "minutely" => logInfo(s"Rolling executor logs enabled for $file with rolling every minute") - Some(60 * 1000L, "--YYYY-MM-dd--HH-mm") + Some(60 * 1000L, "--yyyy-MM-dd--HH-mm") case IntParam(seconds) => logInfo(s"Rolling executor logs enabled for $file with rolling $seconds seconds") - Some(seconds * 1000L, "--YYYY-MM-dd--HH-mm-ss") + Some(seconds * 1000L, "--yyyy-MM-dd--HH-mm-ss") case _ => logWarning(s"Illegal interval for rolling executor logs [$rollingInterval], " + s"rolling logs not enabled") diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala index 84e5c3c917dcb..d7b7219e179d0 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala @@ -109,7 +109,7 @@ private[spark] class SizeBasedRollingPolicy( } @volatile private var bytesWrittenSinceRollover = 0L - val formatter = new SimpleDateFormat("--YYYY-MM-dd--HH-mm-ss--SSSS") + val formatter = new SimpleDateFormat("--yyyy-MM-dd--HH-mm-ss--SSSS") /** Should rollover if the next set of bytes is going to exceed the size limit */ def shouldRollover(bytesToBeWritten: Long): Boolean = { From 9fc373e3a9a8ba7bea9df0950775f48918f63a8a Mon Sep 17 00:00:00 2001 From: Ori Kremer Date: Sun, 22 Jun 2014 20:21:23 -0700 Subject: [PATCH 019/124] SPARK-2241: quote command line args in ec2 script To preserve quoted command line args (in case options have space in them). Author: Ori Kremer Closes #1169 from orikremer/quote_cmd_line_args and squashes the following commits: 67e2aa1 [Ori Kremer] quote command line args --- ec2/spark-ec2 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ec2/spark-ec2 b/ec2/spark-ec2 index 454057aa0d279..31f9771223e51 100755 --- a/ec2/spark-ec2 +++ b/ec2/spark-ec2 @@ -19,4 +19,4 @@ # cd "`dirname $0`" -PYTHONPATH="./third_party/boto-2.4.1.zip/boto-2.4.1:$PYTHONPATH" python ./spark_ec2.py $@ +PYTHONPATH="./third_party/boto-2.4.1.zip/boto-2.4.1:$PYTHONPATH" python ./spark_ec2.py "$@" From 9cb64b2c54b35eed373f54c2103f679b04e9af1e Mon Sep 17 00:00:00 2001 From: Jean-Martin Archer Date: Sun, 22 Jun 2014 20:52:02 -0700 Subject: [PATCH 020/124] SPARK-2166 - Listing of instances to be terminated before the prompt Will list the EC2 instances before detroying the cluster. This was added because it can be scary to destroy EC2 instances without knowing which one will be impacted. Author: Jean-Martin Archer This patch had conflicts when merged, resolved by Committer: Patrick Wendell Closes #270 from j-martin/master and squashes the following commits: 826455f [Jean-Martin Archer] [SPARK-2611] Implementing recommendations 27b0a36 [Jean-Martin Archer] Listing of instances to be terminated before the prompt Will list the EC2 instances before detroying the cluster. This was added because it can be scary to destroy EC2 instances without knowing which one will be impacted. --- ec2/spark_ec2.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index a40311d9fcf02..e80f928b8bebb 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -770,12 +770,16 @@ def real_main(): setup_cluster(conn, master_nodes, slave_nodes, opts, True) elif action == "destroy": - response = raw_input("Are you sure you want to destroy the cluster " + - cluster_name + "?\nALL DATA ON ALL NODES WILL BE LOST!!\n" + - "Destroy cluster " + cluster_name + " (y/N): ") + print "Are you sure you want to destroy the cluster %s?" % cluster_name + print "The following instances will be terminated:" + (master_nodes, slave_nodes) = get_existing_cluster( + conn, opts, cluster_name, die_on_error=False) + for inst in master_nodes + slave_nodes: + print "> %s" % inst.public_dns_name + + msg = "ALL DATA ON ALL NODES WILL BE LOST!!\nDestroy cluster %s (y/N): " % cluster_name + response = raw_input(msg) if response == "y": - (master_nodes, slave_nodes) = get_existing_cluster( - conn, opts, cluster_name, die_on_error=False) print "Terminating master..." for inst in master_nodes: inst.terminate() From e380767de344fd6898429de43da592658fd86a39 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 23 Jun 2014 08:51:11 -0500 Subject: [PATCH 021/124] [SPARK-1395] Fix "local:" URI support in Yarn mode (again). Recent changes ignored the fact that path may be defined with "local:" URIs, which means they need to be explicitly added to the classpath everywhere a remote process is started. This change fixes that by: - Using the correct methods to add paths to the classpath - Creating SparkConf settings for the Spark jar itself and for the user's jar - Propagating those two settings to the remote processes where needed This ensures that both in client and in cluster mode, the driver has the necessary info to build the executor's classpath and have things still work when they contain "local:" references. The change also fixes some confusion in ClientBase about whether to use SparkConf or system properties to propagate config options to the driver and executors, by standardizing on using data held by SparkConf. On the cleanup front, I removed the hacky way that log4j configuration was being propagated to handle the "local:" case. It's much more cleanly (and generically) handled by using spark-submit arguments (--files to upload a config file, or setting spark.executor.extraJavaOptions to pass JVM arguments and use a local file). Author: Marcelo Vanzin Closes #560 from vanzin/yarn-local-2 and squashes the following commits: 4e7f066 [Marcelo Vanzin] Correctly propagate SPARK_JAVA_OPTS to driver/executor. 6a454ea [Marcelo Vanzin] Use constants for PWD in test. 6dd5943 [Marcelo Vanzin] Fix propagation of config options to driver / executor. b2e377f [Marcelo Vanzin] Review feedback. 93c3f85 [Marcelo Vanzin] Fix ClassCastException in test. e5c682d [Marcelo Vanzin] Fix cluster mode, restore SPARK_LOG4J_CONF. 1dfbb40 [Marcelo Vanzin] Add documentation for spark.yarn.jar. bbdce05 [Marcelo Vanzin] [SPARK-1395] Fix "local:" URI support in Yarn mode (again). --- docs/running-on-yarn.md | 28 +- .../apache/spark/deploy/yarn/ClientBase.scala | 248 +++++++++++------- .../deploy/yarn/ExecutorRunnableUtil.scala | 11 +- .../spark/deploy/yarn/ClientBaseSuite.scala | 90 ++++++- .../spark/deploy/yarn/ExecutorRunnable.scala | 1 + 5 files changed, 274 insertions(+), 104 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index fecd8f2cc2d48..43b06df99d9c7 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -95,10 +95,19 @@ Most of the configs are the same for Spark on YARN as for other deployment modes The amount of off heap memory (in megabytes) to be allocated per driver. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. + + spark.yarn.jar + (none) + + The location of the Spark jar file, in case overriding the default location is desired. + By default, Spark on YARN will use a Spark jar installed locally, but the Spark jar can also be + in a world-readable location on HDFS. This allows YARN to cache it on nodes so that it doesn't + need to be distributed each time an application runs. To point to a jar on HDFS, for example, + set this configuration to "hdfs:///some/path". + + -By default, Spark on YARN will use a Spark jar installed locally, but the Spark JAR can also be in a world-readable location on HDFS. This allows YARN to cache it on nodes so that it doesn't need to be distributed each time an application runs. To point to a JAR on HDFS, `export SPARK_JAR=hdfs:///some/path`. - # Launching Spark on YARN Ensure that `HADOOP_CONF_DIR` or `YARN_CONF_DIR` points to the directory which contains the (client side) configuration files for the Hadoop cluster. @@ -156,7 +165,20 @@ all environment variables used for launching each container. This process is use classpath problems in particular. (Note that enabling this requires admin privileges on cluster settings and a restart of all node managers. Thus, this is not applicable to hosted clusters). -# Important Notes +To use a custom log4j configuration for the application master or executors, there are two options: + +- upload a custom log4j.properties using spark-submit, by adding it to the "--files" list of files + to be uploaded with the application. +- add "-Dlog4j.configuration=" to "spark.driver.extraJavaOptions" + (for the driver) or "spark.executor.extraJavaOptions" (for executors). Note that if using a file, + the "file:" protocol should be explicitly provided, and the file needs to exist locally on all + the nodes. + +Note that for the first option, both executors and the application master will share the same +log4j configuration, which may cause issues when they run on the same node (e.g. trying to write +to the same log file). + +# Important notes - Before Hadoop 2.2, YARN does not support cores in container resource requests. Thus, when running against an earlier version, the numbers of cores given via command line arguments cannot be passed to YARN. Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. - The local directories used by Spark executors will be the local directories configured for YARN (Hadoop YARN config `yarn.nodemanager.local-dirs`). If the user specifies `spark.local.dir`, it will be ignored. diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 8f2267599914c..556f49342977a 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -154,7 +154,7 @@ trait ClientBase extends Logging { } /** Copy the file into HDFS if needed. */ - private def copyRemoteFile( + private[yarn] def copyRemoteFile( dstDir: Path, originalPath: Path, replication: Short, @@ -213,10 +213,19 @@ trait ClientBase extends Logging { val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() - Map( - ClientBase.SPARK_JAR -> ClientBase.getSparkJar, ClientBase.APP_JAR -> args.userJar, - ClientBase.LOG4J_PROP -> System.getenv(ClientBase.LOG4J_CONF_ENV_KEY) - ).foreach { case(destName, _localPath) => + val oldLog4jConf = Option(System.getenv("SPARK_LOG4J_CONF")) + if (oldLog4jConf.isDefined) { + logWarning( + "SPARK_LOG4J_CONF detected in the system environment. This variable has been " + + "deprecated. Please refer to the \"Launching Spark on YARN\" documentation " + + "for alternatives.") + } + + List( + (ClientBase.SPARK_JAR, ClientBase.sparkJar(sparkConf), ClientBase.CONF_SPARK_JAR), + (ClientBase.APP_JAR, args.userJar, ClientBase.CONF_SPARK_USER_JAR), + ("log4j.properties", oldLog4jConf.getOrElse(null), null) + ).foreach { case(destName, _localPath, confKey) => val localPath: String = if (_localPath != null) _localPath.trim() else "" if (! localPath.isEmpty()) { val localURI = new URI(localPath) @@ -225,6 +234,8 @@ trait ClientBase extends Logging { val destPath = copyRemoteFile(dst, qualifyForLocal(localURI), replication, setPermissions) distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, destName, statCache) + } else if (confKey != null) { + sparkConf.set(confKey, localPath) } } } @@ -246,6 +257,8 @@ trait ClientBase extends Logging { if (addToClasspath) { cachedSecondaryJarLinks += linkname } + } else if (addToClasspath) { + cachedSecondaryJarLinks += file.trim() } } } @@ -265,14 +278,10 @@ trait ClientBase extends Logging { val env = new HashMap[String, String]() val extraCp = sparkConf.getOption("spark.driver.extraClassPath") - val log4jConf = System.getenv(ClientBase.LOG4J_CONF_ENV_KEY) - ClientBase.populateClasspath(yarnConf, sparkConf, log4jConf, env, extraCp) + ClientBase.populateClasspath(args, yarnConf, sparkConf, env, extraCp) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() - if (log4jConf != null) { - env(ClientBase.LOG4J_CONF_ENV_KEY) = log4jConf - } // Set the environment variables to be passed on to the executors. distCacheMgr.setDistFilesEnv(env) @@ -285,7 +294,6 @@ trait ClientBase extends Logging { // Pass SPARK_YARN_USER_ENV itself to the AM so it can use it to set up executor environments. env("SPARK_YARN_USER_ENV") = userEnvs } - env } @@ -310,6 +318,37 @@ trait ClientBase extends Logging { logInfo("Setting up container launch context") val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) amContainer.setLocalResources(localResources) + + // In cluster mode, if the deprecated SPARK_JAVA_OPTS is set, we need to propagate it to + // executors. But we can't just set spark.executor.extraJavaOptions, because the driver's + // SparkContext will not let that set spark* system properties, which is expected behavior for + // Yarn clients. So propagate it through the environment. + // + // Note that to warn the user about the deprecation in cluster mode, some code from + // SparkConf#validateSettings() is duplicated here (to avoid triggering the condition + // described above). + if (args.amClass == classOf[ApplicationMaster].getName) { + sys.env.get("SPARK_JAVA_OPTS").foreach { value => + val warning = + s""" + |SPARK_JAVA_OPTS was detected (set to '$value'). + |This is deprecated in Spark 1.0+. + | + |Please instead use: + | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application + | - ./spark-submit with --driver-java-options to set -X options for a driver + | - spark.executor.extraJavaOptions to set -X options for executors + """.stripMargin + logWarning(warning) + for (proc <- Seq("driver", "executor")) { + val key = s"spark.$proc.extraJavaOptions" + if (sparkConf.contains(key)) { + throw new SparkException(s"Found both $key and SPARK_JAVA_OPTS. Use only the former.") + } + } + env("SPARK_JAVA_OPTS") = value + } + } amContainer.setEnvironment(env) val amMemory = calculateAMMemory(newApp) @@ -341,30 +380,20 @@ trait ClientBase extends Logging { javaOpts += "-XX:CMSIncrementalDutyCycle=10" } - // SPARK_JAVA_OPTS is deprecated, but for backwards compatibility: - sys.env.get("SPARK_JAVA_OPTS").foreach { opts => - sparkConf.set("spark.executor.extraJavaOptions", opts) - sparkConf.set("spark.driver.extraJavaOptions", opts) - } - + // Forward the Spark configuration to the application master / executors. // TODO: it might be nicer to pass these as an internal environment variable rather than // as Java options, due to complications with string parsing of nested quotes. - if (args.amClass == classOf[ExecutorLauncher].getName) { - // If we are being launched in client mode, forward the spark-conf options - // onto the executor launcher - for ((k, v) <- sparkConf.getAll) { - javaOpts += "-D" + k + "=" + "\\\"" + v + "\\\"" - } - } else { - // If we are being launched in standalone mode, capture and forward any spark - // system properties (e.g. set by spark-class). - for ((k, v) <- sys.props.filterKeys(_.startsWith("spark"))) { - javaOpts += "-D" + k + "=" + "\\\"" + v + "\\\"" - } - sys.props.get("spark.driver.extraJavaOptions").foreach(opts => javaOpts += opts) - sys.props.get("spark.driver.libraryPath").foreach(p => javaOpts += s"-Djava.library.path=$p") + for ((k, v) <- sparkConf.getAll) { + javaOpts += "-D" + k + "=" + "\\\"" + v + "\\\"" + } + + if (args.amClass == classOf[ApplicationMaster].getName) { + sparkConf.getOption("spark.driver.extraJavaOptions") + .orElse(sys.env.get("SPARK_JAVA_OPTS")) + .foreach(opts => javaOpts += opts) + sparkConf.getOption("spark.driver.libraryPath") + .foreach(p => javaOpts += s"-Djava.library.path=$p") } - javaOpts += ClientBase.getLog4jConfiguration(localResources) // Command for the ApplicationMaster val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ @@ -377,7 +406,10 @@ trait ClientBase extends Logging { "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") - logInfo("Command for starting the Spark ApplicationMaster: " + commands) + logInfo("Yarn AM launch context:") + logInfo(s" class: ${args.amClass}") + logInfo(s" env: $env") + logInfo(s" command: ${commands.mkString(" ")}") // TODO: it would be nicer to just make sure there are no null commands here val printableCommands = commands.map(s => if (s == null) "null" else s).toList @@ -391,12 +423,39 @@ trait ClientBase extends Logging { object ClientBase extends Logging { val SPARK_JAR: String = "__spark__.jar" val APP_JAR: String = "__app__.jar" - val LOG4J_PROP: String = "log4j.properties" - val LOG4J_CONF_ENV_KEY: String = "SPARK_LOG4J_CONF" val LOCAL_SCHEME = "local" + val CONF_SPARK_JAR = "spark.yarn.jar" + /** + * This is an internal config used to propagate the location of the user's jar file to the + * driver/executors. + */ + val CONF_SPARK_USER_JAR = "spark.yarn.user.jar" + /** + * This is an internal config used to propagate the list of extra jars to add to the classpath + * of executors. + */ val CONF_SPARK_YARN_SECONDARY_JARS = "spark.yarn.secondary.jars" + val ENV_SPARK_JAR = "SPARK_JAR" - def getSparkJar = sys.env.get("SPARK_JAR").getOrElse(SparkContext.jarOfClass(this.getClass).head) + /** + * Find the user-defined Spark jar if configured, or return the jar containing this + * class if not. + * + * This method first looks in the SparkConf object for the CONF_SPARK_JAR key, and in the + * user environment if that is not found (for backwards compatibility). + */ + def sparkJar(conf: SparkConf) = { + if (conf.contains(CONF_SPARK_JAR)) { + conf.get(CONF_SPARK_JAR) + } else if (System.getenv(ENV_SPARK_JAR) != null) { + logWarning( + s"$ENV_SPARK_JAR detected in the system environment. This variable has been deprecated " + + s"in favor of the $CONF_SPARK_JAR configuration variable.") + System.getenv(ENV_SPARK_JAR) + } else { + SparkContext.jarOfClass(this.getClass).head + } + } def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) = { val classPathElementsToAdd = getYarnAppClasspath(conf) ++ getMRAppClasspath(conf) @@ -469,71 +528,74 @@ object ClientBase extends Logging { triedDefault.toOption } + def populateClasspath(args: ClientArguments, conf: Configuration, sparkConf: SparkConf, + env: HashMap[String, String], extraClassPath: Option[String] = None) { + extraClassPath.foreach(addClasspathEntry(_, env)) + addClasspathEntry(Environment.PWD.$(), env) + + // Normally the users app.jar is last in case conflicts with spark jars + if (sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean) { + addUserClasspath(args, sparkConf, env) + addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env) + ClientBase.populateHadoopClasspath(conf, env) + } else { + addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env) + ClientBase.populateHadoopClasspath(conf, env) + addUserClasspath(args, sparkConf, env) + } + + // Append all jar files under the working directory to the classpath. + addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + "*", env); + } /** - * Returns the java command line argument for setting up log4j. If there is a log4j.properties - * in the given local resources, it is used, otherwise the SPARK_LOG4J_CONF environment variable - * is checked. + * Adds the user jars which have local: URIs (or alternate names, such as APP_JAR) explicitly + * to the classpath. */ - def getLog4jConfiguration(localResources: HashMap[String, LocalResource]): String = { - var log4jConf = LOG4J_PROP - if (!localResources.contains(log4jConf)) { - log4jConf = System.getenv(LOG4J_CONF_ENV_KEY) match { - case conf: String => - val confUri = new URI(conf) - if (ClientBase.LOCAL_SCHEME.equals(confUri.getScheme())) { - "file://" + confUri.getPath() - } else { - ClientBase.LOG4J_PROP - } - case null => "log4j-spark-container.properties" + private def addUserClasspath(args: ClientArguments, conf: SparkConf, + env: HashMap[String, String]) = { + if (args != null) { + addFileToClasspath(args.userJar, APP_JAR, env) + if (args.addJars != null) { + args.addJars.split(",").foreach { case file: String => + addFileToClasspath(file, null, env) + } } + } else { + val userJar = conf.get(CONF_SPARK_USER_JAR, null) + addFileToClasspath(userJar, APP_JAR, env) + + val cachedSecondaryJarLinks = conf.get(CONF_SPARK_YARN_SECONDARY_JARS, "").split(",") + cachedSecondaryJarLinks.foreach(jar => addFileToClasspath(jar, null, env)) } - " -Dlog4j.configuration=" + log4jConf } - def populateClasspath(conf: Configuration, sparkConf: SparkConf, log4jConf: String, - env: HashMap[String, String], extraClassPath: Option[String] = None) { - - if (log4jConf != null) { - // If a custom log4j config file is provided as a local: URI, add its parent directory to the - // classpath. Note that this only works if the custom config's file name is - // "log4j.properties". - val localPath = getLocalPath(log4jConf) - if (localPath != null) { - val parentPath = new File(localPath).getParent() - YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, parentPath, - File.pathSeparator) + /** + * Adds the given path to the classpath, handling "local:" URIs correctly. + * + * If an alternate name for the file is given, and it's not a "local:" file, the alternate + * name will be added to the classpath (relative to the job's work directory). + * + * If not a "local:" file and no alternate name, the environment is not modified. + * + * @param path Path to add to classpath (optional). + * @param fileName Alternate name for the file (optional). + * @param env Map holding the environment variables. + */ + private def addFileToClasspath(path: String, fileName: String, + env: HashMap[String, String]) : Unit = { + if (path != null) { + scala.util.control.Exception.ignoring(classOf[URISyntaxException]) { + val localPath = getLocalPath(path) + if (localPath != null) { + addClasspathEntry(localPath, env) + return + } } } - - /** Add entry to the classpath. */ - def addClasspathEntry(path: String) = YarnSparkHadoopUtil.addToEnvironment(env, - Environment.CLASSPATH.name, path, File.pathSeparator) - /** Add entry to the classpath. Interpreted as a path relative to the working directory. */ - def addPwdClasspathEntry(entry: String) = - addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + entry) - - extraClassPath.foreach(addClasspathEntry) - - val cachedSecondaryJarLinks = - sparkConf.getOption(CONF_SPARK_YARN_SECONDARY_JARS).getOrElse("").split(",") - .filter(_.nonEmpty) - // Normally the users app.jar is last in case conflicts with spark jars - if (sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean) { - addPwdClasspathEntry(APP_JAR) - cachedSecondaryJarLinks.foreach(addPwdClasspathEntry) - addPwdClasspathEntry(SPARK_JAR) - ClientBase.populateHadoopClasspath(conf, env) - } else { - addPwdClasspathEntry(SPARK_JAR) - ClientBase.populateHadoopClasspath(conf, env) - addPwdClasspathEntry(APP_JAR) - cachedSecondaryJarLinks.foreach(addPwdClasspathEntry) + if (fileName != null) { + addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + fileName, env); } - // Append all class files and jar files under the working directory to the classpath. - addClasspathEntry(Environment.PWD.$()) - addPwdClasspathEntry("*") } /** @@ -547,4 +609,8 @@ object ClientBase extends Logging { null } + private def addClasspathEntry(path: String, env: HashMap[String, String]) = + YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, path, + File.pathSeparator) + } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 43dbb2464f929..4ba7133a959ed 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -55,10 +55,12 @@ trait ExecutorRunnableUtil extends Logging { sys.props.get("spark.executor.extraJavaOptions").foreach { opts => javaOpts += opts } + sys.env.get("SPARK_JAVA_OPTS").foreach { opts => + javaOpts += opts + } javaOpts += "-Djava.io.tmpdir=" + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) - javaOpts += ClientBase.getLog4jConfiguration(localResources) // Certain configs need to be passed here because they are needed before the Executor // registers with the Scheduler and transfers the spark configs. Since the Executor backend @@ -166,13 +168,8 @@ trait ExecutorRunnableUtil extends Logging { def prepareEnvironment: HashMap[String, String] = { val env = new HashMap[String, String]() - val extraCp = sparkConf.getOption("spark.executor.extraClassPath") - val log4jConf = System.getenv(ClientBase.LOG4J_CONF_ENV_KEY) - ClientBase.populateClasspath(yarnConf, sparkConf, log4jConf, env, extraCp) - if (log4jConf != null) { - env(ClientBase.LOG4J_CONF_ENV_KEY) = log4jConf - } + ClientBase.populateClasspath(null, yarnConf, sparkConf, env, extraCp) // Allow users to specify some environment variables YarnSparkHadoopUtil.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"), diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala index 608c6e92624c6..686714dc36488 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala @@ -17,22 +17,31 @@ package org.apache.spark.deploy.yarn +import java.io.File import java.net.URI +import com.google.common.io.Files import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.api.ApplicationConstants.Environment - +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.mockito.Matchers._ +import org.mockito.Mockito._ import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers._ +import org.scalatest.Matchers import scala.collection.JavaConversions._ import scala.collection.mutable.{ HashMap => MutableHashMap } import scala.util.Try +import org.apache.spark.SparkConf +import org.apache.spark.util.Utils -class ClientBaseSuite extends FunSuite { +class ClientBaseSuite extends FunSuite with Matchers { test("default Yarn application classpath") { ClientBase.getDefaultYarnApplicationClasspath should be(Some(Fixtures.knownDefYarnAppCP)) @@ -68,6 +77,67 @@ class ClientBaseSuite extends FunSuite { } } + private val SPARK = "local:/sparkJar" + private val USER = "local:/userJar" + private val ADDED = "local:/addJar1,local:/addJar2,/addJar3" + + test("Local jar URIs") { + val conf = new Configuration() + val sparkConf = new SparkConf().set(ClientBase.CONF_SPARK_JAR, SPARK) + val env = new MutableHashMap[String, String]() + val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) + + ClientBase.populateClasspath(args, conf, sparkConf, env, None) + + val cp = env("CLASSPATH").split(File.pathSeparator) + s"$SPARK,$USER,$ADDED".split(",").foreach({ entry => + val uri = new URI(entry) + if (ClientBase.LOCAL_SCHEME.equals(uri.getScheme())) { + cp should contain (uri.getPath()) + } else { + cp should not contain (uri.getPath()) + } + }) + cp should contain (Environment.PWD.$()) + cp should contain (s"${Environment.PWD.$()}${File.separator}*") + cp should not contain (ClientBase.SPARK_JAR) + cp should not contain (ClientBase.APP_JAR) + } + + test("Jar path propagation through SparkConf") { + val conf = new Configuration() + val sparkConf = new SparkConf().set(ClientBase.CONF_SPARK_JAR, SPARK) + val yarnConf = new YarnConfiguration() + val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) + + val client = spy(new DummyClient(args, conf, sparkConf, yarnConf)) + doReturn(new Path("/")).when(client).copyRemoteFile(any(classOf[Path]), + any(classOf[Path]), anyShort(), anyBoolean()) + + var tempDir = Files.createTempDir(); + try { + client.prepareLocalResources(tempDir.getAbsolutePath()) + sparkConf.getOption(ClientBase.CONF_SPARK_USER_JAR) should be (Some(USER)) + + // The non-local path should be propagated by name only, since it will end up in the app's + // staging dir. + val expected = ADDED.split(",") + .map(p => { + val uri = new URI(p) + if (ClientBase.LOCAL_SCHEME == uri.getScheme()) { + p + } else { + Option(uri.getFragment()).getOrElse(new File(p).getName()) + } + }) + .mkString(",") + + sparkConf.getOption(ClientBase.CONF_SPARK_YARN_SECONDARY_JARS) should be (Some(expected)) + } finally { + Utils.deleteRecursively(tempDir) + } + } + object Fixtures { val knownDefYarnAppCP: Seq[String] = @@ -109,4 +179,18 @@ class ClientBaseSuite extends FunSuite { def getFieldValue[A, B](clazz: Class[_], field: String, defaults: => B)(mapTo: A => B): B = Try(clazz.getField(field)).map(_.get(null).asInstanceOf[A]).toOption.map(mapTo).getOrElse(defaults) + private class DummyClient( + val args: ClientArguments, + val conf: Configuration, + val sparkConf: SparkConf, + val yarnConf: YarnConfiguration) extends ClientBase { + + override def calculateAMMemory(newApp: GetNewApplicationResponse): Int = + throw new UnsupportedOperationException() + + override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = + throw new UnsupportedOperationException() + + } + } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 117b33f466f85..07ba0a4b30bd7 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -81,6 +81,7 @@ class ExecutorRunnable( val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, localResources) + logInfo(s"Setting up executor with environment: $env") logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) From b88238faeed8ba723986cf78d64f84965facb236 Mon Sep 17 00:00:00 2001 From: Vlad Date: Mon, 23 Jun 2014 10:55:49 -0500 Subject: [PATCH 022/124] Fixed small running on YARN docs typo The backslash is needed for multiline command Author: Vlad Closes #1158 from frol/patch-1 and squashes the following commits: e258044 [Vlad] Fixed small running on YARN docs typo --- docs/running-on-yarn.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 43b06df99d9c7..5d8d603aa3e37 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -128,7 +128,7 @@ For example: --num-executors 3 \ --driver-memory 4g \ --executor-memory 2g \ - --executor-cores 1 + --executor-cores 1 \ lib/spark-examples*.jar \ 10 From 853a2b951d4c7f6c6c37f53b465b3c7b77691b7c Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Mon, 23 Jun 2014 11:24:05 -0700 Subject: [PATCH 023/124] Fix mvn detection When mvn is not detected (not in executor's path), 'set -e' causes the detection to terminate the script before the helpful error message can be displayed. Author: Matthew Farrellee Closes #1181 from mattf/master-0 and squashes the following commits: 506549f [Matthew Farrellee] Fix mvn detection --- make-distribution.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index ae52b4976dc25..6f708e0ebff6c 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -89,12 +89,12 @@ if [ -z "$JAVA_HOME" ]; then exit -1 fi -VERSION=$(mvn help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1) -if [ $? != 0 ]; then +if ! which mvn &>/dev/null; then echo -e "You need Maven installed to build Spark." echo -e "Download Maven from https://maven.apache.org/" exit -1; fi +VERSION=$(mvn help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1) JAVA_CMD="$JAVA_HOME"/bin/java JAVA_VERSION=$("$JAVA_CMD" -version 2>&1) From a4bc442ca2c35444de8a33376b6f27c6c2a9003d Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 23 Jun 2014 13:24:33 -0700 Subject: [PATCH 024/124] [SPARK-1669][SQL] Made cacheTable idempotent JIRA issue: [SPARK-1669](https://issues.apache.org/jira/browse/SPARK-1669) Caching the same table multiple times should end up with only 1 in-memory columnar representation of this table. Before: ``` scala> loadTestTable("src") ... scala> cacheTable("src") ... scala> cacheTable("src") ... scala> table("src") ... == Query Plan == InMemoryColumnarTableScan [key#2,value#3], (InMemoryRelation [key#2,value#3], false, (InMemoryColumnarTableScan [key#2,value#3], (InMemoryRelation [key#2,value#3], false, (HiveTableScan [key#2,value#3], (MetastoreRelation default, src, None), None)))) ``` After: ``` scala> loadTestTable("src") ... scala> cacheTable("src") ... scala> cacheTable("src") ... scala> table("src") ... == Query Plan == InMemoryColumnarTableScan [key#2,value#3], (InMemoryRelation [key#2,value#3], false, (HiveTableScan [key#2,value#3], (MetastoreRelation default, src, None), None)) ``` Author: Cheng Lian Closes #1183 from liancheng/spark-1669 and squashes the following commits: 68f8a20 [Cheng Lian] Removed an unused import 51bae90 [Cheng Lian] Made cacheTable idempotent --- .../org/apache/spark/sql/SQLContext.scala | 13 ++++++++---- .../org/apache/spark/sql/SQLQuerySuite.scala | 20 +++++++++++++++++++ 2 files changed, 29 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index c60af28b2a1f3..0bcfbf6f849e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -187,10 +187,15 @@ class SQLContext(@transient val sparkContext: SparkContext) /** Caches the specified table in-memory. */ def cacheTable(tableName: String): Unit = { val currentTable = catalog.lookupRelation(None, tableName) - val useCompression = - sparkContext.conf.getBoolean("spark.sql.inMemoryColumnarStorage.compressed", false) - val asInMemoryRelation = - InMemoryRelation(useCompression, executePlan(currentTable).executedPlan) + val asInMemoryRelation = EliminateAnalysisOperators(currentTable.logicalPlan) match { + case _: InMemoryRelation => + currentTable.logicalPlan + + case _ => + val useCompression = + sparkContext.conf.getBoolean("spark.sql.inMemoryColumnarStorage.compressed", false) + InMemoryRelation(useCompression, executePlan(currentTable).executedPlan) + } catalog.registerTable(None, tableName, asInMemoryRelation) } 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 e9360b0fc7910..cca58c0063b38 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 @@ -17,7 +17,9 @@ package org.apache.spark.sql +import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} import org.apache.spark.sql.test._ /* Implicits */ @@ -405,4 +407,22 @@ class SQLQuerySuite extends QueryTest { clear() } + test("SPARK-1669: cacheTable should be idempotent") { + assume(!table("testData").logicalPlan.isInstanceOf[InMemoryRelation]) + + cacheTable("testData") + EliminateAnalysisOperators(table("testData").logicalPlan) match { + case _: InMemoryRelation => + case _ => + fail("testData should be cached") + } + + cacheTable("testData") + EliminateAnalysisOperators(table("testData").logicalPlan) match { + case InMemoryRelation(_, _, _: InMemoryColumnarTableScan) => + fail("cacheTable is not idempotent") + + case _ => + } + } } From 6dc6722a666ec1afc6fb13e7110907d8f5f9cd9d Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 23 Jun 2014 13:35:09 -0700 Subject: [PATCH 025/124] [SPARK-2118] spark class should complain if tools jar is missing. Author: Prashant Sharma Closes #1068 from ScrapCodes/SPARK-2118/tools-jar-check and squashes the following commits: 29e768b [Prashant Sharma] Code Review 5cb6f7d [Prashant Sharma] [SPARK-2118] spark class should complaing if tools jar is missing. --- bin/spark-class | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/bin/spark-class b/bin/spark-class index cfe363a71da31..60d9657c0ffcd 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -130,6 +130,11 @@ else fi if [[ "$1" =~ org.apache.spark.tools.* ]]; then + if test -z "$SPARK_TOOLS_JAR"; then + echo "Failed to find Spark Tools Jar in $FWDIR/tools/target/scala-$SCALA_VERSION/" 1>&2 + echo "You need to build spark before running $1." 1>&2 + exit 1 + fi CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" fi From 21ddd7d1e9f8e2a726427f32422c31706a20ba3f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 23 Jun 2014 13:53:44 -0700 Subject: [PATCH 026/124] [SPARK-1768] History server enhancements. Two improvements to the history server: - Separate the HTTP handling from history fetching, so that it's easy to add new backends later (thinking about SPARK-1537 in the long run) - Avoid loading all UIs in memory. Do lazy loading instead, keeping a few in memory for faster access. This allows the app limit to go away, since holding just the listing in memory shouldn't be too expensive unless the user has millions of completed apps in the history (at which point I'd expect other issues to arise aside from history server memory usage, such as FileSystem.listStatus() starting to become ridiculously expensive). I also fixed a few minor things along the way which aren't really worth mentioning. I also removed the app's log path from the UI since that information may not even exist depending on which backend is used (even though there is only one now). Author: Marcelo Vanzin Closes #718 from vanzin/hist-server and squashes the following commits: 53620c9 [Marcelo Vanzin] Add mima exclude, fix scaladoc wording. c21f8d8 [Marcelo Vanzin] Feedback: formatting, docs. dd8cc4b [Marcelo Vanzin] Standardize on using spark.history.* configuration. 4da3a52 [Marcelo Vanzin] Remove UI from ApplicationHistoryInfo. 2a7f68d [Marcelo Vanzin] Address review feedback. 4e72c77 [Marcelo Vanzin] Remove comment about ordering. 249bcea [Marcelo Vanzin] Remove offset / count from provider interface. ca5d320 [Marcelo Vanzin] Remove code that deals with unfinished apps. 6e2432f [Marcelo Vanzin] Second round of feedback. b2c570a [Marcelo Vanzin] Make class package-private. 4406f61 [Marcelo Vanzin] Cosmetic change to listing header. e852149 [Marcelo Vanzin] Initialize new app array to expected size. e8026f4 [Marcelo Vanzin] Review feedback. 49d2fd3 [Marcelo Vanzin] Fix a comment. 91e96ca [Marcelo Vanzin] Fix scalastyle issues. 6fbe0d8 [Marcelo Vanzin] Better handle failures when loading app info. eee2f5a [Marcelo Vanzin] Ensure server.stop() is called when shutting down. bda2fa1 [Marcelo Vanzin] Rudimentary paging support for the history UI. b284478 [Marcelo Vanzin] Separate history server from history backend. --- .../history/ApplicationHistoryProvider.scala | 59 ++++ .../deploy/history/FsHistoryProvider.scala | 214 ++++++++++++++ .../spark/deploy/history/HistoryPage.scala | 46 +-- .../spark/deploy/history/HistoryServer.scala | 274 ++++++------------ .../history/HistoryServerArguments.scala | 55 ++-- .../scala/org/apache/spark/ui/WebUI.scala | 1 + docs/monitoring.md | 21 +- project/MimaExcludes.scala | 5 +- sbin/start-history-server.sh | 17 +- 9 files changed, 448 insertions(+), 244 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala new file mode 100644 index 0000000000000..a0e8bd403a41d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -0,0 +1,59 @@ +/* + * 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.deploy.history + +import org.apache.spark.ui.SparkUI + +private[spark] case class ApplicationHistoryInfo( + id: String, + name: String, + startTime: Long, + endTime: Long, + lastUpdated: Long, + sparkUser: String) + +private[spark] abstract class ApplicationHistoryProvider { + + /** + * Returns a list of applications available for the history server to show. + * + * @return List of all know applications. + */ + def getListing(): Seq[ApplicationHistoryInfo] + + /** + * Returns the Spark UI for a specific application. + * + * @param appId The application ID. + * @return The application's UI, or null if application is not found. + */ + def getAppUI(appId: String): SparkUI + + /** + * Called when the server is shutting down. + */ + def stop(): Unit = { } + + /** + * Returns configuration data to be shown in the History Server home page. + * + * @return A map with the configuration data. Data is show in the order returned by the map. + */ + def getConfig(): Map[String, String] = Map() + +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala new file mode 100644 index 0000000000000..a8c9ac072449f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -0,0 +1,214 @@ +/* + * 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.deploy.history + +import java.io.FileNotFoundException + +import scala.collection.mutable + +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.scheduler._ +import org.apache.spark.ui.SparkUI +import org.apache.spark.util.Utils + +private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider + with Logging { + + // Interval between each check for event log updates + private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", + conf.getInt("spark.history.updateInterval", 10)) * 1000 + + private val logDir = conf.get("spark.history.fs.logDirectory", null) + if (logDir == null) { + throw new IllegalArgumentException("Logging directory must be specified.") + } + + private val fs = Utils.getHadoopFileSystem(logDir) + + // A timestamp of when the disk was last accessed to check for log updates + private var lastLogCheckTimeMs = -1L + + // List of applications, in order from newest to oldest. + @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil + + /** + * A background thread that periodically checks for event log updates on disk. + * + * If a log check is invoked manually in the middle of a period, this thread re-adjusts the + * time at which it performs the next log check to maintain the same period as before. + * + * TODO: Add a mechanism to update manually. + */ + private val logCheckingThread = new Thread("LogCheckingThread") { + override def run() = Utils.logUncaughtExceptions { + while (true) { + val now = getMonotonicTimeMs() + if (now - lastLogCheckTimeMs > UPDATE_INTERVAL_MS) { + Thread.sleep(UPDATE_INTERVAL_MS) + } else { + // If the user has manually checked for logs recently, wait until + // UPDATE_INTERVAL_MS after the last check time + Thread.sleep(lastLogCheckTimeMs + UPDATE_INTERVAL_MS - now) + } + checkForLogs() + } + } + } + + initialize() + + private def initialize() { + // Validate the log directory. + val path = new Path(logDir) + if (!fs.exists(path)) { + throw new IllegalArgumentException( + "Logging directory specified does not exist: %s".format(logDir)) + } + if (!fs.getFileStatus(path).isDir) { + throw new IllegalArgumentException( + "Logging directory specified is not a directory: %s".format(logDir)) + } + + checkForLogs() + logCheckingThread.setDaemon(true) + logCheckingThread.start() + } + + override def getListing() = appList + + override def getAppUI(appId: String): SparkUI = { + try { + val appLogDir = fs.getFileStatus(new Path(logDir, appId)) + loadAppInfo(appLogDir, true)._2 + } catch { + case e: FileNotFoundException => null + } + } + + override def getConfig(): Map[String, String] = + Map(("Event Log Location" -> logDir)) + + /** + * Builds the application list based on the current contents of the log directory. + * Tries to reuse as much of the data already in memory as possible, by not reading + * applications that haven't been updated since last time the logs were checked. + */ + private def checkForLogs() = { + lastLogCheckTimeMs = getMonotonicTimeMs() + logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs)) + try { + val logStatus = fs.listStatus(new Path(logDir)) + val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() + val logInfos = logDirs.filter { + dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE)) + } + + val currentApps = Map[String, ApplicationHistoryInfo]( + appList.map(app => (app.id -> app)):_*) + + // For any application that either (i) is not listed or (ii) has changed since the last time + // the listing was created (defined by the log dir's modification time), load the app's info. + // Otherwise just reuse what's already in memory. + val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size) + for (dir <- logInfos) { + val curr = currentApps.getOrElse(dir.getPath().getName(), null) + if (curr == null || curr.lastUpdated < getModificationTime(dir)) { + try { + newApps += loadAppInfo(dir, false)._1 + } catch { + case e: Exception => logError(s"Failed to load app info from directory $dir.") + } + } else { + newApps += curr + } + } + + appList = newApps.sortBy { info => -info.endTime } + } catch { + case t: Throwable => logError("Exception in checking for event log updates", t) + } + } + + /** + * Parse the application's logs to find out the information we need to build the + * listing page. + * + * When creating the listing of available apps, there is no need to load the whole UI for the + * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user + * clicks on a specific application. + * + * @param logDir Directory with application's log files. + * @param renderUI Whether to create the SparkUI for the application. + * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false. + */ + private def loadAppInfo(logDir: FileStatus, renderUI: Boolean) = { + val elogInfo = EventLoggingListener.parseLoggingInfo(logDir.getPath(), fs) + val path = logDir.getPath + val appId = path.getName + val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec) + val appListener = new ApplicationEventListener + replayBus.addListener(appListener) + + val ui: SparkUI = if (renderUI) { + val conf = this.conf.clone() + val appSecManager = new SecurityManager(conf) + new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId) + // Do not call ui.bind() to avoid creating a new server for each application + } else { + null + } + + replayBus.replay() + val appInfo = ApplicationHistoryInfo( + appId, + appListener.appName, + appListener.startTime, + appListener.endTime, + getModificationTime(logDir), + appListener.sparkUser) + + if (ui != null) { + val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false) + ui.getSecurityManager.setUIAcls(uiAclsEnabled) + ui.getSecurityManager.setViewAcls(appListener.sparkUser, appListener.viewAcls) + } + (appInfo, ui) + } + + /** Return when this directory was last modified. */ + private def getModificationTime(dir: FileStatus): Long = { + try { + val logFiles = fs.listStatus(dir.getPath) + if (logFiles != null && !logFiles.isEmpty) { + logFiles.map(_.getModificationTime).max + } else { + dir.getModificationTime + } + } catch { + case t: Throwable => + logError("Exception in accessing modification time of %s".format(dir.getPath), t) + -1L + } + } + + /** Returns the system's mononotically increasing time. */ + private def getMonotonicTimeMs() = System.nanoTime() / (1000 * 1000) + +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 180c853ce3096..a958c837c2ff6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -25,20 +25,36 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { + private val pageSize = 20 + def render(request: HttpServletRequest): Seq[Node] = { - val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated } - val appTable = UIUtils.listingTable(appHeader, appRow, appRows) + val requestedPage = Option(request.getParameter("page")).getOrElse("1").toInt + val requestedFirst = (requestedPage - 1) * pageSize + + val allApps = parent.getApplicationList() + val actualFirst = if (requestedFirst < allApps.size) requestedFirst else 0 + val apps = allApps.slice(actualFirst, Math.min(actualFirst + pageSize, allApps.size)) + + val actualPage = (actualFirst / pageSize) + 1 + val last = Math.min(actualFirst + pageSize, allApps.size) - 1 + val pageCount = allApps.size / pageSize + (if (allApps.size % pageSize > 0) 1 else 0) + + val appTable = UIUtils.listingTable(appHeader, appRow, apps) + val providerConfig = parent.getProviderConfig() val content =
    -
  • Event Log Location: {parent.baseLogDir}
  • + { providerConfig.map(e =>
  • {e._1}: {e._2}
  • ) }
{ - if (parent.appIdToInfo.size > 0) { + if (allApps.size > 0) {

- Showing {parent.appIdToInfo.size}/{parent.getNumApplications} - Completed Application{if (parent.getNumApplications > 1) "s" else ""} + Showing {actualFirst + 1}-{last + 1} of {allApps.size} + + {if (actualPage > 1) <} + {if (actualPage < pageCount) >} +

++ appTable } else { @@ -56,26 +72,20 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { "Completed", "Duration", "Spark User", - "Log Directory", "Last Updated") private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { - val appName = if (info.started) info.name else info.logDirPath.getName - val uiAddress = parent.getAddress + info.ui.basePath - val startTime = if (info.started) UIUtils.formatDate(info.startTime) else "Not started" - val endTime = if (info.completed) UIUtils.formatDate(info.endTime) else "Not completed" - val difference = if (info.started && info.completed) info.endTime - info.startTime else -1L - val duration = if (difference > 0) UIUtils.formatDuration(difference) else "---" - val sparkUser = if (info.started) info.sparkUser else "Unknown user" - val logDirectory = info.logDirPath.getName + val uiAddress = "/history/" + info.id + val startTime = UIUtils.formatDate(info.startTime) + val endTime = UIUtils.formatDate(info.endTime) + val duration = UIUtils.formatDuration(info.endTime - info.startTime) val lastUpdated = UIUtils.formatDate(info.lastUpdated) - {appName} + {info.name} {startTime} {endTime} {duration} - {sparkUser} - {logDirectory} + {info.sparkUser} {lastUpdated} } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index a9c11dca5678e..29a78a56c8ed5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -17,14 +17,15 @@ package org.apache.spark.deploy.history -import scala.collection.mutable +import java.util.NoSuchElementException +import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} -import org.apache.hadoop.fs.{FileStatus, Path} +import com.google.common.cache._ +import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.scheduler._ -import org.apache.spark.ui.{WebUI, SparkUI} +import org.apache.spark.ui.{WebUI, SparkUI, UIUtils} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.Utils @@ -38,56 +39,68 @@ import org.apache.spark.util.Utils * application's event logs are maintained in the application's own sub-directory. This * is the same structure as maintained in the event log write code path in * EventLoggingListener. - * - * @param baseLogDir The base directory in which event logs are found */ class HistoryServer( - val baseLogDir: String, + conf: SparkConf, + provider: ApplicationHistoryProvider, securityManager: SecurityManager, - conf: SparkConf) - extends WebUI(securityManager, HistoryServer.WEB_UI_PORT, conf) with Logging { - - import HistoryServer._ + port: Int) + extends WebUI(securityManager, port, conf) with Logging { - private val fileSystem = Utils.getHadoopFileSystem(baseLogDir) - private val localHost = Utils.localHostName() - private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) + // How many applications to retain + private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50) - // A timestamp of when the disk was last accessed to check for log updates - private var lastLogCheckTime = -1L + private val appLoader = new CacheLoader[String, SparkUI] { + override def load(key: String): SparkUI = { + val ui = provider.getAppUI(key) + if (ui == null) { + throw new NoSuchElementException() + } + attachSparkUI(ui) + ui + } + } - // Number of completed applications found in this directory - private var numCompletedApplications = 0 + private val appCache = CacheBuilder.newBuilder() + .maximumSize(retainedApplications) + .removalListener(new RemovalListener[String, SparkUI] { + override def onRemoval(rm: RemovalNotification[String, SparkUI]) = { + detachSparkUI(rm.getValue()) + } + }) + .build(appLoader) + + private val loaderServlet = new HttpServlet { + protected override def doGet(req: HttpServletRequest, res: HttpServletResponse): Unit = { + val parts = Option(req.getPathInfo()).getOrElse("").split("/") + if (parts.length < 2) { + res.sendError(HttpServletResponse.SC_BAD_REQUEST, + s"Unexpected path info in request (URI = ${req.getRequestURI()}") + return + } - @volatile private var stopped = false + val appId = parts(1) - /** - * A background thread that periodically checks for event log updates on disk. - * - * If a log check is invoked manually in the middle of a period, this thread re-adjusts the - * time at which it performs the next log check to maintain the same period as before. - * - * TODO: Add a mechanism to update manually. - */ - private val logCheckingThread = new Thread { - override def run(): Unit = Utils.logUncaughtExceptions { - while (!stopped) { - val now = System.currentTimeMillis - if (now - lastLogCheckTime > UPDATE_INTERVAL_MS) { - checkForLogs() - Thread.sleep(UPDATE_INTERVAL_MS) - } else { - // If the user has manually checked for logs recently, wait until - // UPDATE_INTERVAL_MS after the last check time - Thread.sleep(lastLogCheckTime + UPDATE_INTERVAL_MS - now) + // Note we don't use the UI retrieved from the cache; the cache loader above will register + // the app's UI, and all we need to do is redirect the user to the same URI that was + // requested, and the proper data should be served at that point. + try { + appCache.get(appId) + res.sendRedirect(res.encodeRedirectURL(req.getRequestURI())) + } catch { + case e: Exception => e.getCause() match { + case nsee: NoSuchElementException => + val msg =
Application {appId} not found.
+ res.setStatus(HttpServletResponse.SC_NOT_FOUND) + UIUtils.basicSparkPage(msg, "Not Found").foreach( + n => res.getWriter().write(n.toString)) + + case cause: Exception => throw cause } } } } - // A mapping of application ID to its history information, which includes the rendered UI - val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]() - initialize() /** @@ -98,108 +111,23 @@ class HistoryServer( */ def initialize() { attachPage(new HistoryPage(this)) - attachHandler(createStaticHandler(STATIC_RESOURCE_DIR, "/static")) + attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) + + val contextHandler = new ServletContextHandler + contextHandler.setContextPath("/history") + contextHandler.addServlet(new ServletHolder(loaderServlet), "/*") + attachHandler(contextHandler) } /** Bind to the HTTP server behind this web interface. */ override def bind() { super.bind() - logCheckingThread.start() - } - - /** - * Check for any updates to event logs in the base directory. This is only effective once - * the server has been bound. - * - * If a new completed application is found, the server renders the associated SparkUI - * from the application's event logs, attaches this UI to itself, and stores metadata - * information for this application. - * - * If the logs for an existing completed application are no longer found, the server - * removes all associated information and detaches the SparkUI. - */ - def checkForLogs() = synchronized { - if (serverInfo.isDefined) { - lastLogCheckTime = System.currentTimeMillis - logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTime)) - try { - val logStatus = fileSystem.listStatus(new Path(baseLogDir)) - val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() - val logInfos = logDirs - .sortBy { dir => getModificationTime(dir) } - .map { dir => (dir, EventLoggingListener.parseLoggingInfo(dir.getPath, fileSystem)) } - .filter { case (dir, info) => info.applicationComplete } - - // Logging information for applications that should be retained - val retainedLogInfos = logInfos.takeRight(RETAINED_APPLICATIONS) - val retainedAppIds = retainedLogInfos.map { case (dir, _) => dir.getPath.getName } - - // Remove any applications that should no longer be retained - appIdToInfo.foreach { case (appId, info) => - if (!retainedAppIds.contains(appId)) { - detachSparkUI(info.ui) - appIdToInfo.remove(appId) - } - } - - // Render the application's UI if it is not already there - retainedLogInfos.foreach { case (dir, info) => - val appId = dir.getPath.getName - if (!appIdToInfo.contains(appId)) { - renderSparkUI(dir, info) - } - } - - // Track the total number of completed applications observed this round - numCompletedApplications = logInfos.size - - } catch { - case e: Exception => logError("Exception in checking for event log updates", e) - } - } else { - logWarning("Attempted to check for event log updates before binding the server.") - } - } - - /** - * Render a new SparkUI from the event logs if the associated application is completed. - * - * HistoryServer looks for a special file that indicates application completion in the given - * directory. If this file exists, the associated application is regarded to be completed, in - * which case the server proceeds to render the SparkUI. Otherwise, the server does nothing. - */ - private def renderSparkUI(logDir: FileStatus, elogInfo: EventLoggingInfo) { - val path = logDir.getPath - val appId = path.getName - val replayBus = new ReplayListenerBus(elogInfo.logPaths, fileSystem, elogInfo.compressionCodec) - val appListener = new ApplicationEventListener - replayBus.addListener(appListener) - val appConf = conf.clone() - val appSecManager = new SecurityManager(appConf) - val ui = new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId) - - // Do not call ui.bind() to avoid creating a new server for each application - replayBus.replay() - if (appListener.applicationStarted) { - appSecManager.setUIAcls(HISTORY_UI_ACLS_ENABLED) - appSecManager.setViewAcls(appListener.sparkUser, appListener.viewAcls) - attachSparkUI(ui) - val appName = appListener.appName - val sparkUser = appListener.sparkUser - val startTime = appListener.startTime - val endTime = appListener.endTime - val lastUpdated = getModificationTime(logDir) - ui.setAppName(appName + " (completed)") - appIdToInfo(appId) = ApplicationHistoryInfo(appId, appName, startTime, endTime, - lastUpdated, sparkUser, path, ui) - } } /** Stop the server and close the file system. */ override def stop() { super.stop() - stopped = true - fileSystem.close() + provider.stop() } /** Attach a reconstructed UI to this server. Only valid after bind(). */ @@ -215,27 +143,20 @@ class HistoryServer( ui.getHandlers.foreach(detachHandler) } - /** Return the address of this server. */ - def getAddress: String = "http://" + publicHost + ":" + boundPort + /** + * Returns a list of available applications, in descending order according to their end time. + * + * @return List of all known applications. + */ + def getApplicationList() = provider.getListing() - /** Return the number of completed applications found, whether or not the UI is rendered. */ - def getNumApplications: Int = numCompletedApplications + /** + * Returns the provider configuration to show in the listing page. + * + * @return A map with the provider's configuration. + */ + def getProviderConfig() = provider.getConfig() - /** Return when this directory was last modified. */ - private def getModificationTime(dir: FileStatus): Long = { - try { - val logFiles = fileSystem.listStatus(dir.getPath) - if (logFiles != null && !logFiles.isEmpty) { - logFiles.map(_.getModificationTime).max - } else { - dir.getModificationTime - } - } catch { - case e: Exception => - logError("Exception in accessing modification time of %s".format(dir.getPath), e) - -1L - } - } } /** @@ -251,30 +172,31 @@ class HistoryServer( object HistoryServer { private val conf = new SparkConf - // Interval between each check for event log updates - val UPDATE_INTERVAL_MS = conf.getInt("spark.history.updateInterval", 10) * 1000 - - // How many applications to retain - val RETAINED_APPLICATIONS = conf.getInt("spark.history.retainedApplications", 250) - - // The port to which the web UI is bound - val WEB_UI_PORT = conf.getInt("spark.history.ui.port", 18080) - - // set whether to enable or disable view acls for all applications - val HISTORY_UI_ACLS_ENABLED = conf.getBoolean("spark.history.ui.acls.enable", false) - - val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR - def main(argStrings: Array[String]) { initSecurity() - val args = new HistoryServerArguments(argStrings) + val args = new HistoryServerArguments(conf, argStrings) val securityManager = new SecurityManager(conf) - val server = new HistoryServer(args.logDir, securityManager, conf) + + val providerName = conf.getOption("spark.history.provider") + .getOrElse(classOf[FsHistoryProvider].getName()) + val provider = Class.forName(providerName) + .getConstructor(classOf[SparkConf]) + .newInstance(conf) + .asInstanceOf[ApplicationHistoryProvider] + + val port = conf.getInt("spark.history.ui.port", 18080) + + val server = new HistoryServer(conf, provider, securityManager, port) server.bind() + Runtime.getRuntime().addShutdownHook(new Thread("HistoryServerStopper") { + override def run() = { + server.stop() + } + }) + // Wait until the end of the world... or if the HistoryServer process is manually stopped while(true) { Thread.sleep(Int.MaxValue) } - server.stop() } def initSecurity() { @@ -291,17 +213,3 @@ object HistoryServer { } } - - -private[spark] case class ApplicationHistoryInfo( - id: String, - name: String, - startTime: Long, - endTime: Long, - lastUpdated: Long, - sparkUser: String, - logDirPath: Path, - ui: SparkUI) { - def started = startTime != -1 - def completed = endTime != -1 -} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index 943c061743dbd..be9361b754fc3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -17,17 +17,14 @@ package org.apache.spark.deploy.history -import java.net.URI - -import org.apache.hadoop.fs.Path - +import org.apache.spark.SparkConf import org.apache.spark.util.Utils /** * Command-line parser for the master. */ -private[spark] class HistoryServerArguments(args: Array[String]) { - var logDir = "" +private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String]) { + private var logDir: String = null parse(args.toList) @@ -45,32 +42,36 @@ private[spark] class HistoryServerArguments(args: Array[String]) { case _ => printUsageAndExit(1) } - validateLogDir() - } - - private def validateLogDir() { - if (logDir == "") { - System.err.println("Logging directory must be specified.") - printUsageAndExit(1) - } - val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) - val path = new Path(logDir) - if (!fileSystem.exists(path)) { - System.err.println("Logging directory specified does not exist: %s".format(logDir)) - printUsageAndExit(1) - } - if (!fileSystem.getFileStatus(path).isDir) { - System.err.println("Logging directory specified is not a directory: %s".format(logDir)) - printUsageAndExit(1) + if (logDir != null) { + conf.set("spark.history.fs.logDirectory", logDir) } } private def printUsageAndExit(exitCode: Int) { System.err.println( - "Usage: HistoryServer [options]\n" + - "\n" + - "Options:\n" + - " -d DIR, --dir DIR Location of event log files") + """ + |Usage: HistoryServer + | + |Configuration options can be set by setting the corresponding JVM system property. + |History Server options are always available; additional options depend on the provider. + | + |History Server options: + | + | spark.history.ui.port Port where server will listen for connections + | (default 18080) + | spark.history.acls.enable Whether to enable view acls for all applications + | (default false) + | spark.history.provider Name of history provider class (defaults to + | file system-based provider) + | spark.history.retainedApplications Max number of application UIs to keep loaded in memory + | (default 50) + |FsHistoryProvider options: + | + | spark.history.fs.logDirectory Directory where app logs are stored (required) + | spark.history.fs.updateInterval How often to reload log data from storage (in seconds, + | default 10) + |""".stripMargin) System.exit(exitCode) } + } diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index b08f308fda1dd..856273e1d4e21 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -51,6 +51,7 @@ private[spark] abstract class WebUI( def getTabs: Seq[WebUITab] = tabs.toSeq def getHandlers: Seq[ServletContextHandler] = handlers.toSeq + def getSecurityManager: SecurityManager = securityManager /** Attach a tab to this UI, along with all of its attached pages. */ def attachTab(tab: WebUITab) { diff --git a/docs/monitoring.md b/docs/monitoring.md index 2b9e9e5bd7ea0..84073fe4d949a 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -35,11 +35,13 @@ If Spark is run on Mesos or YARN, it is still possible to reconstruct the UI of application through Spark's history server, provided that the application's event logs exist. You can start a the history server by executing: - ./sbin/start-history-server.sh + ./sbin/start-history-server.sh -The base logging directory must be supplied, and should contain sub-directories that each -represents an application's event logs. This creates a web interface at -`http://:18080` by default. The history server can be configured as follows: +When using the file-system provider class (see spark.history.provider below), the base logging +directory must be supplied in the spark.history.fs.logDirectory configuration option, +and should contain sub-directories that each represents an application's event logs. This creates a +web interface at `http://:18080` by default. The history server can be configured as +follows: @@ -69,7 +71,14 @@ represents an application's event logs. This creates a web interface at
Environment VariableMeaning
- + + + + + + - + }} } 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 a3f824a4e1f57..30971f769682f 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 @@ -91,13 +91,13 @@ private[ui] class StageTableBase( {s.name} - val details = if (s.details.nonEmpty) ( + val details = if (s.details.nonEmpty) { +show details - ) + } listener.stageIdToDescription.get(s.stageId) .map(d =>
{d}
{nameLink} {killLink}
) From 62d4a0fa9947e64c1533f66ae577557bcfb271c9 Mon Sep 17 00:00:00 2001 From: Zichuan Ye Date: Thu, 26 Jun 2014 15:21:29 -0700 Subject: [PATCH 057/124] Fixing AWS instance type information based upon current EC2 data Fixed a problem in previous file in which some information regarding AWS instance types were wrong. Such information was updated base upon current AWS EC2 data. Author: Zichuan Ye Closes #1156 from jerry86/master and squashes the following commits: ff36e95 [Zichuan Ye] Fixing AWS instance type information based upon current EC2 data --- ec2/spark_ec2.py | 25 +++++++++++++++++-------- 1 file changed, 17 insertions(+), 8 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index e80f928b8bebb..e22d93bd31bc2 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -203,6 +203,8 @@ def get_spark_shark_version(opts): # Attempt to resolve an appropriate AMI given the architecture and # region of the request. +# Information regarding Amazon Linux AMI instance type was update on 2014-6-20: +# http://aws.amazon.com/amazon-linux-ami/instance-type-matrix/ def get_spark_ami(opts): instance_types = { "m1.small": "pvm", @@ -218,10 +220,12 @@ def get_spark_ami(opts): "cc1.4xlarge": "hvm", "cc2.8xlarge": "hvm", "cg1.4xlarge": "hvm", - "hs1.8xlarge": "hvm", - "hi1.4xlarge": "hvm", - "m3.xlarge": "hvm", - "m3.2xlarge": "hvm", + "hs1.8xlarge": "pvm", + "hi1.4xlarge": "pvm", + "m3.medium": "pvm", + "m3.large": "pvm", + "m3.xlarge": "pvm", + "m3.2xlarge": "pvm", "cr1.8xlarge": "hvm", "i2.xlarge": "hvm", "i2.2xlarge": "hvm", @@ -526,7 +530,8 @@ def wait_for_cluster(conn, wait_secs, master_nodes, slave_nodes): # Get number of local disks available for a given EC2 instance type. def get_num_disks(instance_type): - # From http://docs.amazonwebservices.com/AWSEC2/latest/UserGuide/index.html?InstanceStorage.html + # From http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/InstanceStorage.html + # Updated 2014-6-20 disks_by_instance = { "m1.small": 1, "m1.medium": 1, @@ -544,8 +549,10 @@ def get_num_disks(instance_type): "hs1.8xlarge": 24, "cr1.8xlarge": 2, "hi1.4xlarge": 2, - "m3.xlarge": 0, - "m3.2xlarge": 0, + "m3.medium": 1, + "m3.large": 1, + "m3.xlarge": 2, + "m3.2xlarge": 2, "i2.xlarge": 1, "i2.2xlarge": 2, "i2.4xlarge": 4, @@ -559,7 +566,9 @@ def get_num_disks(instance_type): "r3.xlarge": 1, "r3.2xlarge": 1, "r3.4xlarge": 1, - "r3.8xlarge": 2 + "r3.8xlarge": 2, + "g2.2xlarge": 1, + "t1.micro": 0 } if instance_type in disks_by_instance: return disks_by_instance[instance_type] From f1f7385a5087a80c936d419699e3f5232455f189 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 26 Jun 2014 17:09:24 -0700 Subject: [PATCH 058/124] Strip '@' symbols when merging pull requests. Currently all of the commits with 'X' in them cause person X to receive e-mails every time someone makes a public fork of Spark. marmbrus who requested this. Author: Patrick Wendell Closes #1239 from pwendell/strip and squashes the following commits: 22e5a97 [Patrick Wendell] Strip '@' symbols when merging pull requests. --- dev/merge_spark_pr.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index ffb70096d6014..c44320239bbbf 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -130,7 +130,9 @@ def merge_pr(pr_num, target_ref): merge_message_flags += ["-m", title] if body != None: - merge_message_flags += ["-m", body] + # We remove @ symbols from the body to avoid triggering e-mails + # to people every time someone creates a public fork of Spark. + merge_message_flags += ["-m", body.replace("@", "")] authors = "\n".join(["Author: %s" % a for a in distinct_authors]) From 981bde9b056ef5e91aed553e0b5930f12e1ff797 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 26 Jun 2014 19:18:11 -0700 Subject: [PATCH 059/124] [SQL]Extract the joinkeys from join condition Extract the join keys from equality conditions, that can be evaluated using equi-join. Author: Cheng Hao Closes #1190 from chenghao-intel/extract_join_keys and squashes the following commits: 4a1060a [Cheng Hao] Fix some of the small issues ceb4924 [Cheng Hao] Remove the redundant pattern of join keys extraction cec34e8 [Cheng Hao] Update the code style issues dcc4584 [Cheng Hao] Extract the joinkeys from join condition --- .../sql/catalyst/optimizer/Optimizer.scala | 7 ++- .../sql/catalyst/planning/patterns.scala | 62 +++++++------------ .../spark/sql/execution/SparkStrategies.scala | 13 ++-- 3 files changed, 33 insertions(+), 49 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index b20b5de8c46eb..fb517e40677ec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -257,8 +257,11 @@ object PushPredicateThroughProject extends Rule[LogicalPlan] { * Check https://cwiki.apache.org/confluence/display/Hive/OuterJoinBehavior for more details */ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { - // split the condition expression into 3 parts, - // (canEvaluateInLeftSide, canEvaluateInRightSide, haveToEvaluateWithBothSide) + /** + * Splits join condition expressions into three categories based on the attributes required + * to evaluate them. + * @returns (canEvaluateInLeft, canEvaluateInRight, haveToEvaluateInBoth) + */ private def split(condition: Seq[Expression], left: LogicalPlan, right: LogicalPlan) = { val (leftEvaluateCondition, rest) = condition.partition(_.references subsetOf left.outputSet) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index a43bef389c4bf..026692abe067d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -105,57 +105,39 @@ object PhysicalOperation extends PredicateHelper { } /** - * A pattern that finds joins with equality conditions that can be evaluated using hashing - * techniques. For inner joins, any filters on top of the join operator are also matched. + * A pattern that finds joins with equality conditions that can be evaluated using equi-join. */ -object HashFilteredJoin extends Logging with PredicateHelper { +object ExtractEquiJoinKeys extends Logging with PredicateHelper { /** (joinType, rightKeys, leftKeys, condition, leftChild, rightChild) */ type ReturnType = (JoinType, Seq[Expression], Seq[Expression], Option[Expression], LogicalPlan, LogicalPlan) def unapply(plan: LogicalPlan): Option[ReturnType] = plan match { - // All predicates can be evaluated for inner join (i.e., those that are in the ON - // clause and WHERE clause.) - case FilteredOperation(predicates, join @ Join(left, right, Inner, condition)) => - logger.debug(s"Considering hash inner join on: ${predicates ++ condition}") - splitPredicates(predicates ++ condition, join) - // All predicates can be evaluated for left semi join (those that are in the WHERE - // clause can only from left table, so they can all be pushed down.) - case FilteredOperation(predicates, join @ Join(left, right, LeftSemi, condition)) => - logger.debug(s"Considering hash left semi join on: ${predicates ++ condition}") - splitPredicates(predicates ++ condition, join) case join @ Join(left, right, joinType, condition) => - logger.debug(s"Considering hash join on: $condition") - splitPredicates(condition.toSeq, join) - case _ => None - } - - // Find equi-join predicates that can be evaluated before the join, and thus can be used - // as join keys. - def splitPredicates(allPredicates: Seq[Expression], join: Join): Option[ReturnType] = { - val Join(left, right, joinType, _) = join - val (joinPredicates, otherPredicates) = - allPredicates.flatMap(splitConjunctivePredicates).partition { - case EqualTo(l, r) if (canEvaluate(l, left) && canEvaluate(r, right)) || - (canEvaluate(l, right) && canEvaluate(r, left)) => true - case _ => false + logger.debug(s"Considering join on: $condition") + // Find equi-join predicates that can be evaluated before the join, and thus can be used + // as join keys. + val (joinPredicates, otherPredicates) = + condition.map(splitConjunctivePredicates).getOrElse(Nil).partition { + case EqualTo(l, r) if (canEvaluate(l, left) && canEvaluate(r, right)) || + (canEvaluate(l, right) && canEvaluate(r, left)) => true + case _ => false + } + + val joinKeys = joinPredicates.map { + case EqualTo(l, r) if canEvaluate(l, left) && canEvaluate(r, right) => (l, r) + case EqualTo(l, r) if canEvaluate(l, right) && canEvaluate(r, left) => (r, l) } - - val joinKeys = joinPredicates.map { - case EqualTo(l, r) if canEvaluate(l, left) && canEvaluate(r, right) => (l, r) - case EqualTo(l, r) if canEvaluate(l, right) && canEvaluate(r, left) => (r, l) - } - - // Do not consider this strategy if there are no join keys. - if (joinKeys.nonEmpty) { val leftKeys = joinKeys.map(_._1) val rightKeys = joinKeys.map(_._2) - Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right)) - } else { - logger.debug(s"Avoiding hash join with no join keys.") - None - } + if (joinKeys.nonEmpty) { + logger.debug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}") + Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right)) + } else { + None + } + case _ => None } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 3cd29967d1cd5..0925605b7c4d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -31,9 +31,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object LeftSemiJoin extends Strategy with PredicateHelper { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - // Find left semi joins where at least some predicates can be evaluated by matching hash - // keys using the HashFilteredJoin pattern. - case HashFilteredJoin(LeftSemi, leftKeys, rightKeys, condition, left, right) => + // Find left semi joins where at least some predicates can be evaluated by matching join keys + case ExtractEquiJoinKeys(LeftSemi, leftKeys, rightKeys, condition, left, right) => val semiJoin = execution.LeftSemiJoinHash( leftKeys, rightKeys, planLater(left), planLater(right)) condition.map(Filter(_, semiJoin)).getOrElse(semiJoin) :: Nil @@ -46,7 +45,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } /** - * Uses the HashFilteredJoin pattern to find joins where at least some of the predicates can be + * Uses the ExtractEquiJoinKeys pattern to find joins where at least some of the predicates can be * evaluated by matching hash keys. */ object HashJoin extends Strategy with PredicateHelper { @@ -65,7 +64,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def broadcastTables: Seq[String] = sqlContext.joinBroadcastTables.split(",").toBuffer def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case HashFilteredJoin( + case ExtractEquiJoinKeys( Inner, leftKeys, rightKeys, @@ -75,7 +74,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { if broadcastTables.contains(b.tableName) => broadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildRight) - case HashFilteredJoin( + case ExtractEquiJoinKeys( Inner, leftKeys, rightKeys, @@ -85,7 +84,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { if broadcastTables.contains(b.tableName) => broadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildLeft) - case HashFilteredJoin(Inner, leftKeys, rightKeys, condition, left, right) => + case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) => val hashJoin = execution.ShuffledHashJoin( leftKeys, rightKeys, BuildRight, planLater(left), planLater(right)) From bf578deaf2493081ceeb78dfd7617def5699a06e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 26 Jun 2014 21:12:16 -0700 Subject: [PATCH 060/124] Removed throwable field from FetchFailedException and added MetadataFetchFailedException FetchFailedException used to have a Throwable field, but in reality we never propagate any of the throwable/exceptions back to the driver because Executor explicitly looks for FetchFailedException and then sends FetchFailed as the TaskEndReason. This pull request removes the throwable and adds a MetadataFetchFailedException that extends FetchFailedException (so now MapOutputTracker throws MetadataFetchFailedException instead). Author: Reynold Xin Closes #1227 from rxin/metadataFetchException and squashes the following commits: 5cb1e0a [Reynold Xin] MetadataFetchFailedException extends FetchFailedException. 8861ee2 [Reynold Xin] Throw MetadataFetchFailedException in MapOutputTracker. --- .../org/apache/spark/MapOutputTracker.scala | 12 +++--- .../org/apache/spark/TaskEndReason.scala | 2 +- .../org/apache/spark/executor/Executor.scala | 2 +- .../spark/scheduler/TaskDescription.scala | 4 ++ .../{ => shuffle}/FetchFailedException.scala | 43 +++++++++++-------- .../hash/BlockStoreShuffleFetcher.scala | 5 ++- .../apache/spark/MapOutputTrackerSuite.scala | 1 + 7 files changed, 42 insertions(+), 27 deletions(-) rename core/src/main/scala/org/apache/spark/{ => shuffle}/FetchFailedException.scala (50%) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index ee82d9fa7874b..182abacc475ae 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -25,9 +25,11 @@ import scala.concurrent.Await import akka.actor._ import akka.pattern.ask + +import org.apache.spark.util._ import org.apache.spark.scheduler.MapStatus +import org.apache.spark.shuffle.MetadataFetchFailedException import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util._ private[spark] sealed trait MapOutputTrackerMessage private[spark] case class GetMapOutputStatuses(shuffleId: Int) @@ -168,8 +170,8 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, fetchedStatuses) } } else { - throw new FetchFailedException(null, shuffleId, -1, reduceId, - new Exception("Missing all output locations for shuffle " + shuffleId)) + throw new MetadataFetchFailedException( + shuffleId, reduceId, "Missing all output locations for shuffle " + shuffleId) } } else { statuses.synchronized { @@ -371,8 +373,8 @@ private[spark] object MapOutputTracker { statuses.map { status => if (status == null) { - throw new FetchFailedException(null, shuffleId, -1, reduceId, - new Exception("Missing an output location for shuffle " + shuffleId)) + throw new MetadataFetchFailedException( + shuffleId, reduceId, "Missing an output location for shuffle " + shuffleId) } else { (status.location, decompressSize(status.compressedSizes(reduceId))) } diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 5e8bd8c8e533a..df42d679b4699 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -65,7 +65,7 @@ case object Resubmitted extends TaskFailedReason { */ @DeveloperApi case class FetchFailed( - bmAddress: BlockManagerId, + bmAddress: BlockManagerId, // Note that bmAddress can be null shuffleId: Int, mapId: Int, reduceId: Int) 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 557b9a3f46a08..4d3ba11633bf5 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -26,8 +26,8 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler._ +import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} import org.apache.spark.util.{AkkaUtils, Utils} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala index 1481d70db42e1..4c96b9e5fef60 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala @@ -21,6 +21,10 @@ import java.nio.ByteBuffer import org.apache.spark.util.SerializableBuffer +/** + * Description of a task that gets passed onto executors to be executed, usually created by + * [[TaskSetManager.resourceOffer]]. + */ private[spark] class TaskDescription( val taskId: Long, val executorId: String, diff --git a/core/src/main/scala/org/apache/spark/FetchFailedException.scala b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala similarity index 50% rename from core/src/main/scala/org/apache/spark/FetchFailedException.scala rename to core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala index 8eaa26bdb1b5b..71c08e9d5a8c3 100644 --- a/core/src/main/scala/org/apache/spark/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala @@ -15,31 +15,38 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.shuffle import org.apache.spark.storage.BlockManagerId +import org.apache.spark.{FetchFailed, TaskEndReason} +/** + * Failed to fetch a shuffle block. The executor catches this exception and propagates it + * back to DAGScheduler (through TaskEndReason) so we'd resubmit the previous stage. + * + * Note that bmAddress can be null. + */ private[spark] class FetchFailedException( - taskEndReason: TaskEndReason, - message: String, - cause: Throwable) + bmAddress: BlockManagerId, + shuffleId: Int, + mapId: Int, + reduceId: Int) extends Exception { - def this (bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int, - cause: Throwable) = - this(FetchFailed(bmAddress, shuffleId, mapId, reduceId), - "Fetch failed: %s %d %d %d".format(bmAddress, shuffleId, mapId, reduceId), - cause) - - def this (shuffleId: Int, reduceId: Int, cause: Throwable) = - this(FetchFailed(null, shuffleId, -1, reduceId), - "Unable to fetch locations from master: %d %d".format(shuffleId, reduceId), cause) - - override def getMessage(): String = message + override def getMessage: String = + "Fetch failed: %s %d %d %d".format(bmAddress, shuffleId, mapId, reduceId) + def toTaskEndReason: TaskEndReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId) +} - override def getCause(): Throwable = cause - - def toTaskEndReason: TaskEndReason = taskEndReason +/** + * Failed to get shuffle metadata from [[org.apache.spark.MapOutputTracker]]. + */ +private[spark] class MetadataFetchFailedException( + shuffleId: Int, + reduceId: Int, + message: String) + extends FetchFailedException(null, shuffleId, -1, reduceId) { + override def getMessage: String = message } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index b05b6ea345df3..a932455776e34 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -20,11 +20,12 @@ package org.apache.spark.shuffle.hash import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap +import org.apache.spark._ import org.apache.spark.executor.ShuffleReadMetrics import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId} import org.apache.spark.util.CompletionIterator -import org.apache.spark._ private[hash] object BlockStoreShuffleFetcher extends Logging { def fetch[T]( @@ -63,7 +64,7 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { blockId match { case ShuffleBlockId(shufId, mapId, _) => val address = statuses(mapId.toInt)._1 - throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId, null) + throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId) case _ => throw new SparkException( "Failed to get block " + blockId + ", which is not a shuffle block") diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 95ba273f16a71..9702838085627 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -24,6 +24,7 @@ import akka.testkit.TestActorRef import org.scalatest.FunSuite import org.apache.spark.scheduler.MapStatus +import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AkkaUtils From d1636dd72fc4966413baeb97ba55b313dc1da63d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 26 Jun 2014 21:13:26 -0700 Subject: [PATCH 061/124] [SPARK-2297][UI] Make task attempt and speculation more explicit in UI. New UI: ![screen shot 2014-06-26 at 1 43 52 pm](https://cloud.githubusercontent.com/assets/323388/3404643/82b9ddc6-fd73-11e3-96f9-f7592a7aee79.png) Author: Reynold Xin Closes #1236 from rxin/ui-task-attempt and squashes the following commits: 3b645dd [Reynold Xin] Expose attemptId in Stage. c0474b1 [Reynold Xin] Beefed up unit test. c404bdd [Reynold Xin] Fix ReplayListenerSuite. f56be4b [Reynold Xin] Fixed JsonProtocolSuite. e29e0f7 [Reynold Xin] Minor update. 5e4354a [Reynold Xin] [SPARK-2297][UI] Make task attempt and speculation more explicit in UI. --- .../spark/scheduler/SparkListener.scala | 2 + .../org/apache/spark/scheduler/Stage.scala | 2 + .../org/apache/spark/scheduler/TaskInfo.scala | 4 +- .../spark/scheduler/TaskSetManager.scala | 24 +++-- .../org/apache/spark/ui/jobs/StagePage.scala | 11 ++- .../org/apache/spark/util/JsonProtocol.scala | 9 +- .../ui/jobs/JobProgressListenerSuite.scala | 11 +-- .../apache/spark/util/JsonProtocolSuite.scala | 88 ++++++++++++------- 8 files changed, 102 insertions(+), 49 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 378cf1aaebe7b..82163eadd56e9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -75,9 +75,11 @@ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) @DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent +@DeveloperApi case class SparkListenerApplicationStart(appName: String, time: Long, sparkUser: String) extends SparkListenerEvent +@DeveloperApi case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ 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 9a4be43ee219f..8ec482a6f6d9c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -106,6 +106,8 @@ private[spark] class Stage( id } + def attemptId: Int = nextAttemptId + val name = callSite.short val details = callSite.long diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 4c62e4dc0bac8..6aecdfe8e6656 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -27,10 +27,12 @@ import org.apache.spark.annotation.DeveloperApi class TaskInfo( val taskId: Long, val index: Int, + val attempt: Int, val launchTime: Long, val executorId: String, val host: String, - val taskLocality: TaskLocality.TaskLocality) { + val taskLocality: TaskLocality.TaskLocality, + val speculative: Boolean) { /** * The time when the task started remotely getting the result. Will not be set if the diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index c0898f64fb0c9..83ff6b8550b4f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -335,17 +335,19 @@ private[spark] class TaskSetManager( /** * Dequeue a pending task for a given node and return its index and locality level. * Only search for tasks matching the given locality constraint. + * + * @return An option containing (task index within the task set, locality, is speculative?) */ private def findTask(execId: String, host: String, locality: TaskLocality.Value) - : Option[(Int, TaskLocality.Value)] = + : Option[(Int, TaskLocality.Value, Boolean)] = { for (index <- findTaskFromList(execId, getPendingTasksForExecutor(execId))) { - return Some((index, TaskLocality.PROCESS_LOCAL)) + return Some((index, TaskLocality.PROCESS_LOCAL, false)) } if (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) { for (index <- findTaskFromList(execId, getPendingTasksForHost(host))) { - return Some((index, TaskLocality.NODE_LOCAL)) + return Some((index, TaskLocality.NODE_LOCAL, false)) } } @@ -354,23 +356,25 @@ private[spark] class TaskSetManager( rack <- sched.getRackForHost(host) index <- findTaskFromList(execId, getPendingTasksForRack(rack)) } { - return Some((index, TaskLocality.RACK_LOCAL)) + return Some((index, TaskLocality.RACK_LOCAL, false)) } } // Look for no-pref tasks after rack-local tasks since they can run anywhere. for (index <- findTaskFromList(execId, pendingTasksWithNoPrefs)) { - return Some((index, TaskLocality.PROCESS_LOCAL)) + return Some((index, TaskLocality.PROCESS_LOCAL, false)) } if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { for (index <- findTaskFromList(execId, allPendingTasks)) { - return Some((index, TaskLocality.ANY)) + return Some((index, TaskLocality.ANY, false)) } } // Finally, if all else has failed, find a speculative task - findSpeculativeTask(execId, host, locality) + findSpeculativeTask(execId, host, locality).map { case (taskIndex, allowedLocality) => + (taskIndex, allowedLocality, true) + } } /** @@ -391,7 +395,7 @@ private[spark] class TaskSetManager( } findTask(execId, host, allowedLocality) match { - case Some((index, taskLocality)) => { + case Some((index, taskLocality, speculative)) => { // Found a task; do some bookkeeping and return a task description val task = tasks(index) val taskId = sched.newTaskId() @@ -400,7 +404,9 @@ private[spark] class TaskSetManager( taskSet.id, index, taskId, execId, host, taskLocality)) // Do various bookkeeping copiesRunning(index) += 1 - val info = new TaskInfo(taskId, index, curTime, execId, host, taskLocality) + val attemptNum = taskAttempts(index).size + val info = new TaskInfo( + taskId, index, attemptNum + 1, curTime, execId, host, taskLocality, speculative) taskInfos(taskId) = info taskAttempts(index) = info :: taskAttempts(index) // Update our locality level for delay scheduling diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 8b65f0671bdb9..8e3d5d1cd4c6b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -95,8 +95,9 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { // scalastyle:on val taskHeaders: Seq[String] = - Seq("Task Index", "Task ID", "Status", "Locality Level", "Executor", "Launch Time") ++ - Seq("Duration", "GC Time", "Result Ser Time") ++ + Seq( + "Index", "ID", "Attempt", "Status", "Locality Level", "Executor", + "Launch Time", "Duration", "GC Time") ++ {if (hasShuffleRead) Seq("Shuffle Read") else Nil} ++ {if (hasShuffleWrite) Seq("Write Time", "Shuffle Write") else Nil} ++ {if (hasBytesSpilled) Seq("Shuffle Spill (Memory)", "Shuffle Spill (Disk)") else Nil} ++ @@ -245,6 +246,9 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { + @@ -255,9 +259,12 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { + {if (shuffleRead) { @@ -94,10 +95,11 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { - - - + + + + // scalastyle:on } /** Represent an executor's info as a map given a storage status index */ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index c83e196c9c156..add0e9878a546 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -67,18 +67,20 @@ private[ui] class ExecutorTable(stageId: Int, parent: JobProgressTab) { executorIdToSummary match { case Some(x) => x.toSeq.sortBy(_._1).map { case (k, v) => { + // scalastyle:off - + - - - - + + + + + // scalastyle:on } } case _ => Seq[Node]() diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index b66edd91f56c0..9813d9330ac7f 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -49,6 +49,7 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { /** Render an HTML row representing an RDD */ private def rddRow(rdd: RDDInfo): Seq[Node] = { + // scalastyle:off - - - + + + + // scalastyle:on } } From 21e0f77b6321590ed86223a60cdb8ae08ea4057f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 27 Jun 2014 15:23:25 -0700 Subject: [PATCH 064/124] [SPARK-2307] SparkUI - storage tab displays incorrect RDDs The issue here is that the `StorageTab` listens for updates from the `StorageStatusListener`, but when a block is kicked out of the cache, `StorageStatusListener` removes it from its list. Thus, there is no way for the `StorageTab` to know whether a block has been dropped. This issue was introduced in #1080, which was itself a bug fix. Here we revert that PR and offer a different fix for the original bug (SPARK-2144). Author: Andrew Or Closes #1249 from andrewor14/storage-ui-fix and squashes the following commits: af019ce [Andrew Or] Fix SPARK-2307 --- .../org/apache/spark/storage/StorageStatusListener.scala | 6 +----- .../main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala | 5 ++++- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index c694fc8c347ec..a6e6627d54e01 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -37,11 +37,7 @@ class StorageStatusListener extends SparkListener { val filteredStatus = storageStatusList.find(_.blockManagerId.executorId == execId) filteredStatus.foreach { storageStatus => updatedBlocks.foreach { case (blockId, updatedStatus) => - if (updatedStatus.storageLevel == StorageLevel.NONE) { - storageStatus.blocks.remove(blockId) - } else { - storageStatus.blocks(blockId) = updatedStatus - } + storageStatus.blocks(blockId) = updatedStatus } } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 2d8c3b949c1ac..6cfc46c7e78fe 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -21,6 +21,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node +import org.apache.spark.storage.StorageLevel import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils @@ -107,7 +108,9 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { val status = listener.storageStatusList(statusId) val execId = status.blockManagerId.executorId val hostPort = status.blockManagerId.hostPort - val rddBlocks = status.blocks.size + val rddBlocks = status.blocks.count { case (_, blockStatus) => + blockStatus.storageLevel != StorageLevel.NONE + } val memUsed = status.memUsed val maxMem = status.maxMem val diskUsed = status.diskUsed From f17510e371dfbeaada3c72b884d70c36503ea30a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 27 Jun 2014 16:11:31 -0700 Subject: [PATCH 065/124] [SPARK-2259] Fix highly misleading docs on cluster / client deploy modes The existing docs are highly misleading. For standalone mode, for example, it encourages the user to use standalone-cluster mode, which is not officially supported. The safeguards have been added in Spark submit itself to prevent bad documentation from leading users down the wrong path in the future. This PR is prompted by countless headaches users of Spark have run into on the mailing list. Author: Andrew Or Closes #1200 from andrewor14/submit-docs and squashes the following commits: 5ea2460 [Andrew Or] Rephrase cluster vs client explanation c827f32 [Andrew Or] Clarify spark submit messages 9f7ed8f [Andrew Or] Clarify client vs cluster deploy mode + add safeguards --- .../org/apache/spark/deploy/SparkSubmit.scala | 17 ++++++++++++++--- .../spark/deploy/SparkSubmitArguments.scala | 5 +++-- docs/running-on-mesos.md | 3 ++- docs/spark-standalone.md | 9 ++++----- docs/submitting-applications.md | 14 +++++++++++++- 5 files changed, 36 insertions(+), 12 deletions(-) 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 7e9a9344e61f9..b050dccb6d57f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy import java.io.{File, PrintStream} import java.lang.reflect.InvocationTargetException -import java.net.{URI, URL} +import java.net.URL import scala.collection.mutable.{ArrayBuffer, HashMap, Map} @@ -117,14 +117,25 @@ object SparkSubmit { val isPython = args.isPython val isYarnCluster = clusterManager == YARN && deployOnCluster + // For mesos, only client mode is supported if (clusterManager == MESOS && deployOnCluster) { - printErrorAndExit("Cannot currently run driver on the cluster in Mesos") + printErrorAndExit("Cluster deploy mode is currently not supported for Mesos clusters.") + } + + // For standalone, only client mode is supported + if (clusterManager == STANDALONE && deployOnCluster) { + printErrorAndExit("Cluster deploy mode is currently not supported for standalone clusters.") + } + + // For shells, only client mode is applicable + if (isShell(args.primaryResource) && deployOnCluster) { + printErrorAndExit("Cluster deploy mode is not applicable to Spark shells.") } // If we're running a python app, set the main class to our specific python runner if (isPython) { if (deployOnCluster) { - printErrorAndExit("Cannot currently run Python driver programs on cluster") + printErrorAndExit("Cluster deploy mode is currently not supported for python.") } if (args.primaryResource == PYSPARK_SHELL) { args.mainClass = "py4j.GatewayServer" 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 f1032ea8dbada..57655aa4c32b1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -338,8 +338,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { """Usage: spark-submit [options] [app options] |Options: | --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. - | --deploy-mode DEPLOY_MODE Where to run the driver program: either "client" to run - | on the local machine, or "cluster" to run inside cluster. + | --deploy-mode DEPLOY_MODE Whether to launch the driver program locally ("client") or + | on one of the worker machines inside the cluster ("cluster") + | (Default: client). | --class CLASS_NAME Your application's main class (for Java / Scala apps). | --name NAME A name of your application. | --jars JARS Comma-separated list of local jars to include on the driver diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index e3c8922404365..bd046cfc1837d 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -127,7 +127,8 @@ val sc = new SparkContext(conf) {% endhighlight %} (You can also use [`spark-submit`](submitting-applications.html) and configure `spark.executor.uri` -in the [conf/spark-defaults.conf](configuration.html#loading-default-configurations) file.) +in the [conf/spark-defaults.conf](configuration.html#loading-default-configurations) file. Note +that `spark-submit` currently only supports deploying the Spark driver in `client` mode for Mesos.) When running a shell, the `spark.executor.uri` parameter is inherited from `SPARK_EXECUTOR_URI`, so it does not need to be redundantly passed in as a system property. diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 3c1ce06083ede..f5c0f7cef83d2 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -235,11 +235,10 @@ You can also pass an option `--cores ` to control the number of cores # Launching Compiled Spark Applications -Spark supports two deploy modes: applications may run with the driver inside the client process or -entirely inside the cluster. The -[`spark-submit` script](submitting-applications.html) provides the -most straightforward way to submit a compiled Spark application to the cluster in either deploy -mode. +The [`spark-submit` script](submitting-applications.html) provides the most straightforward way to +submit a compiled Spark application to the cluster. For standalone clusters, Spark currently +only supports deploying the driver inside the client process that is submitting the application +(`client` deploy mode). If your application is launched through Spark submit, then the application jar is automatically distributed to all worker nodes. For any additional jars that your application depends on, you diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index d2864fe4c2f65..e05883072bfa8 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -42,10 +42,22 @@ Some of the commonly used options are: * `--class`: The entry point for your application (e.g. `org.apache.spark.examples.SparkPi`) * `--master`: The [master URL](#master-urls) for the cluster (e.g. `spark://23.195.26.187:7077`) -* `--deploy-mode`: Whether to deploy your driver program within the cluster or run it locally as an external client (either `cluster` or `client`) +* `--deploy-mode`: Whether to deploy your driver on the worker nodes (`cluster`) or locally as an external client (`client`) (default: `client`)* * `application-jar`: Path to a bundled jar including your application and all dependencies. The URL must be globally visible inside of your cluster, for instance, an `hdfs://` path or a `file://` path that is present on all nodes. * `application-arguments`: Arguments passed to the main method of your main class, if any +*A common deployment strategy is to submit your application from a gateway machine that is +physically co-located with your worker machines (e.g. Master node in a standalone EC2 cluster). +In this setup, `client` mode is appropriate. In `client` mode, the driver is launched directly +within the client `spark-submit` process, with the input and output of the application attached +to the console. Thus, this mode is especially suitable for applications that involve the REPL +(e.g. Spark shell). + +Alternatively, if your application is submitted from a machine far from the worker machines (e.g. +locally on your laptop), it is common to use `cluster` mode to minimize network latency between +the drivers and the executors. Note that `cluster` mode is currently not supported for standalone +clusters, Mesos clusters, or python applications. + For Python applications, simply pass a `.py` file in the place of `` instead of a JAR, and add Python `.zip`, `.egg` or `.py` files to the search path with `--py-files`. From 0e0686d3ef88e024fcceafe36a0cdbb953f5aeae Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Fri, 27 Jun 2014 18:20:33 -0700 Subject: [PATCH 066/124] [SPARK-2003] Fix python SparkContext example Author: Matthew Farrellee Closes #1246 from mattf/SPARK-2003 and squashes the following commits: b12e7ca [Matthew Farrellee] [SPARK-2003] Fix python SparkContext example --- docs/programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 65d75b85efda6..06e4c4ce527e1 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -145,7 +145,7 @@ that contains information about your application. {% highlight python %} conf = SparkConf().setAppName(appName).setMaster(master) -sc = SparkContext(conf) +sc = SparkContext(conf=conf) {% endhighlight %} From b8f2e13aec715e038bd6d1d07b607683f138ac83 Mon Sep 17 00:00:00 2001 From: Guillaume Ballet Date: Sat, 28 Jun 2014 13:07:12 -0700 Subject: [PATCH 067/124] [SPARK-2233] make-distribution script should list the git hash in the RELEASE file This patch adds the git revision hash (short version) to the RELEASE file. It uses git instead of simply checking for the existence of .git, so as to make sure that this is a functional repository. Author: Guillaume Ballet Closes #1216 from gballet/master and squashes the following commits: eabc50f [Guillaume Ballet] Refactored the script to take comments into account. d93e5e8 [Guillaume Ballet] [SPARK 2233] make-distribution script now lists the git hash tag in the RELEASE file. --- make-distribution.sh | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/make-distribution.sh b/make-distribution.sh index 86868438e75c3..94b473bf91cd3 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -100,6 +100,14 @@ if [ -z "$JAVA_HOME" ]; then exit -1 fi +if which git &>/dev/null; then + GITREV=$(git rev-parse --short HEAD 2>/dev/null || :) + if [ ! -z $GITREV ]; then + GITREVSTRING=" (git revision $GITREV)" + fi + unset GITREV +fi + if ! which mvn &>/dev/null; then echo -e "You need Maven installed to build Spark." echo -e "Download Maven from https://maven.apache.org/" @@ -186,7 +194,7 @@ ${BUILD_COMMAND} # Make directories rm -rf "$DISTDIR" mkdir -p "$DISTDIR/lib" -echo "Spark $VERSION built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE" +echo "Spark $VERSION$GITREVSTRING built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE" # Copy jars cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" From 3c104c79d24425786cec0034f269ba19cf465b31 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Sat, 28 Jun 2014 18:39:27 -0700 Subject: [PATCH 068/124] [SPARK-1394] Remove SIGCHLD handler in worker subprocess It should not be the responsibility of the worker subprocess, which does not intentionally fork, to try and cleanup child processes. Doing so is complex and interferes with operations such as platform.system(). If it is desirable to have tighter control over subprocesses, then namespaces should be used and it should be the manager's resposibility to handle cleanup. Author: Matthew Farrellee Closes #1247 from mattf/SPARK-1394 and squashes the following commits: c36f308 [Matthew Farrellee] [SPARK-1394] Remove SIGCHLD handler in worker subprocess --- python/pyspark/daemon.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index b2f226a55ec13..5eb1c63bf206b 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -103,6 +103,7 @@ def waitSocketClose(sock): if os.fork() == 0: # Leave the worker pool signal.signal(SIGHUP, SIG_DFL) + signal.signal(SIGCHLD, SIG_DFL) listen_sock.close() # Read the socket using fdopen instead of socket.makefile() because the latter # seems to be very slow; note that we need to dup() the file descriptor because From 2053d793cc2e8e5f5776e6576ddc6f8e6168e60c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 28 Jun 2014 21:05:03 -0700 Subject: [PATCH 069/124] Improve MapOutputTracker error logging. Author: Reynold Xin Closes #1258 from rxin/mapOutputTracker and squashes the following commits: a7c95b6 [Reynold Xin] Improve MapOutputTracker error logging. --- .../org/apache/spark/MapOutputTracker.scala | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 182abacc475ae..894091761485d 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -26,10 +26,10 @@ import scala.concurrent.Await import akka.actor._ import akka.pattern.ask -import org.apache.spark.util._ import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle.MetadataFetchFailedException import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util._ private[spark] sealed trait MapOutputTrackerMessage private[spark] case class GetMapOutputStatuses(shuffleId: Int) @@ -107,14 +107,17 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging Await.result(future, timeout) } catch { case e: Exception => + logError("Error communicating with MapOutputTracker", e) throw new SparkException("Error communicating with MapOutputTracker", e) } } /** Send a one-way message to the trackerActor, to which we expect it to reply with true. */ protected def sendTracker(message: Any) { - if (askTracker(message) != true) { - throw new SparkException("Error reply received from MapOutputTracker") + val response = askTracker(message) + if (response != true) { + throw new SparkException( + "Error reply received from MapOutputTracker. Expecting true, got " + response.toString) } } @@ -366,9 +369,9 @@ private[spark] object MapOutputTracker { // any of the statuses is null (indicating a missing location due to a failed mapper), // throw a FetchFailedException. private def convertMapStatuses( - shuffleId: Int, - reduceId: Int, - statuses: Array[MapStatus]): Array[(BlockManagerId, Long)] = { + shuffleId: Int, + reduceId: Int, + statuses: Array[MapStatus]): Array[(BlockManagerId, Long)] = { assert (statuses != null) statuses.map { status => @@ -403,7 +406,7 @@ private[spark] object MapOutputTracker { if (compressedSize == 0) { 0 } else { - math.pow(LOG_BASE, (compressedSize & 0xFF)).toLong + math.pow(LOG_BASE, compressedSize & 0xFF).toLong } } } From cdf613fc52d7057555a2dbf2241ce90bacbabb4a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 29 Jun 2014 16:46:28 -0700 Subject: [PATCH 070/124] [SPARK-2320] Reduce exception/code block font size in web ui Author: Reynold Xin Closes #1261 from rxin/ui-pre-size and squashes the following commits: 7ab1a69 [Reynold Xin] [SPARK-2320] Reduce exception/code block font size in web ui --- core/src/main/resources/org/apache/spark/ui/static/webui.css | 4 ++++ core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index a8bc141208a94..11fd956bfbe66 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -95,6 +95,10 @@ span.expand-details { float: right; } +pre { + font-size: 0.8em; +} + .stage-details { max-height: 100px; overflow-y: auto; 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 30971f769682f..b2b6cc6a6ec2d 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 @@ -168,7 +168,7 @@ private[ui] class FailedStageTable( override protected def stageRow(s: StageInfo): Seq[Node] = { val basicColumns = super.stageRow(s) - val failureReason = + val failureReason = basicColumns ++ failureReason } } From 7b71a0e09622e09285a9884ebb67b5fb1c5caa53 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Sun, 29 Jun 2014 22:01:42 -0700 Subject: [PATCH 071/124] [SPARK-1683] Track task read metrics. This commit adds a new metric in TaskMetrics to record the input data size and displays this information in the UI. An earlier version of this commit also added the read time, which can be useful for diagnosing straggler problems, but unfortunately that change introduced a significant performance regression for jobs that don't do much computation. In order to track read time, we'll need to do sampling. The screenshots below show the UI with the new "Input" field, which I added to the stage summary page, the executor summary page, and the per-stage page. ![image](https://cloud.githubusercontent.com/assets/1108612/3167930/2627f92a-eb77-11e3-861c-98ea5bb7a1a2.png) ![image](https://cloud.githubusercontent.com/assets/1108612/3167936/475a889c-eb77-11e3-9706-f11c48751f17.png) ![image](https://cloud.githubusercontent.com/assets/1108612/3167948/80ebcf12-eb77-11e3-87ed-349fce6a770c.png) Author: Kay Ousterhout Closes #962 from kayousterhout/read_metrics and squashes the following commits: f13b67d [Kay Ousterhout] Correctly format input bytes on executor page 8b70cde [Kay Ousterhout] Added comment about potential inaccuracy of bytesRead d1016e8 [Kay Ousterhout] Udated SparkListenerSuite test 8461492 [Kay Ousterhout] Miniscule style fix ae04d99 [Kay Ousterhout] Remove input metrics for parallel collections 719f19d [Kay Ousterhout] Style fixes bb6ec62 [Kay Ousterhout] Small fixes 869ac7b [Kay Ousterhout] Updated Json tests 44a0301 [Kay Ousterhout] Fixed accidentally added line 4bd0568 [Kay Ousterhout] Added input source, renamed Hdfs to Hadoop. f27e535 [Kay Ousterhout] Updates based on review comments and to fix rebase bf41029 [Kay Ousterhout] Updated Json tests to pass 0fc33e0 [Kay Ousterhout] Added explicit backward compatibility test 4e52925 [Kay Ousterhout] Added Json output and associated tests. 365400b [Kay Ousterhout] [SPARK-1683] Track task read metrics. --- .../scala/org/apache/spark/CacheManager.scala | 10 +- .../apache/spark/executor/TaskMetrics.scala | 29 +++++ .../scala/org/apache/spark/rdd/BlockRDD.scala | 2 +- .../org/apache/spark/rdd/HadoopRDD.scala | 15 +++ .../org/apache/spark/rdd/NewHadoopRDD.scala | 13 +++ .../apache/spark/scheduler/JobLogger.scala | 15 ++- .../apache/spark/storage/BlockManager.scala | 63 ++++++----- .../apache/spark/storage/ThreadingTest.scala | 2 +- .../apache/spark/ui/exec/ExecutorsPage.scala | 4 + .../apache/spark/ui/exec/ExecutorsTab.scala | 5 + .../spark/ui/jobs/ExecutorSummary.scala | 1 + .../apache/spark/ui/jobs/ExecutorTable.scala | 2 + .../spark/ui/jobs/JobProgressListener.scala | 15 ++- .../org/apache/spark/ui/jobs/StagePage.scala | 41 +++++-- .../org/apache/spark/ui/jobs/StageTable.scala | 7 ++ .../org/apache/spark/util/JsonProtocol.scala | 20 +++- .../org/apache/spark/CacheManagerSuite.scala | 4 +- .../spark/scheduler/SparkListenerSuite.scala | 1 + .../spark/storage/BlockManagerSuite.scala | 84 +++++++++++---- .../apache/spark/util/JsonProtocolSuite.scala | 102 +++++++++++++++--- 20 files changed, 349 insertions(+), 86 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 3f667a4a0f9c5..8f867686a0443 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -19,6 +19,7 @@ package org.apache.spark import scala.collection.mutable.{ArrayBuffer, HashSet} +import org.apache.spark.executor.InputMetrics import org.apache.spark.rdd.RDD import org.apache.spark.storage._ @@ -41,9 +42,10 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { val key = RDDBlockId(rdd.id, partition.index) logDebug(s"Looking for partition $key") blockManager.get(key) match { - case Some(values) => + case Some(blockResult) => // Partition is already materialized, so just return its values - new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) + context.taskMetrics.inputMetrics = Some(blockResult.inputMetrics) + new InterruptibleIterator(context, blockResult.data.asInstanceOf[Iterator[T]]) case None => // Acquire a lock for loading this partition @@ -110,7 +112,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { logInfo(s"Whoever was loading $id failed; we'll try it ourselves") loading.add(id) } - values.map(_.asInstanceOf[Iterator[T]]) + values.map(_.data.asInstanceOf[Iterator[T]]) } } } @@ -132,7 +134,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { * exceptions that can be avoided. */ updatedBlocks ++= blockManager.put(key, values, storageLevel, tellMaster = true) blockManager.get(key) match { - case Some(v) => v.asInstanceOf[Iterator[T]] + case Some(v) => v.data.asInstanceOf[Iterator[T]] case None => logInfo(s"Failure to store $key") throw new BlockException(key, s"Block manager failed to return cached value for $key!") diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 350fd74173f65..ac73288442a74 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -66,6 +66,12 @@ class TaskMetrics extends Serializable { */ var diskBytesSpilled: Long = _ + /** + * If this task reads from a HadoopRDD or from persisted data, metrics on how much data was read + * are stored here. + */ + var inputMetrics: Option[InputMetrics] = None + /** * If this task reads from shuffle output, metrics on getting shuffle data will be collected here */ @@ -87,6 +93,29 @@ private[spark] object TaskMetrics { def empty: TaskMetrics = new TaskMetrics } +/** + * :: DeveloperApi :: + * Method by which input data was read. Network means that the data was read over the network + * from a remote block manager (which may have stored the data on-disk or in-memory). + */ +@DeveloperApi +object DataReadMethod extends Enumeration with Serializable { + type DataReadMethod = Value + val Memory, Disk, Hadoop, Network = Value +} + +/** + * :: DeveloperApi :: + * Metrics about reading input data. + */ +@DeveloperApi +case class InputMetrics(readMethod: DataReadMethod.Value) { + /** + * Total bytes read. + */ + var bytesRead: Long = 0L +} + /** * :: DeveloperApi :: diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index c64da8804d166..2673ec22509e9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -46,7 +46,7 @@ class BlockRDD[T: ClassTag](@transient sc: SparkContext, @transient val blockIds val blockManager = SparkEnv.get.blockManager val blockId = split.asInstanceOf[BlockRDDPartition].blockId blockManager.get(blockId) match { - case Some(block) => block.asInstanceOf[Iterator[T]] + case Some(block) => block.data.asInstanceOf[Iterator[T]] case None => throw new Exception("Could not compute split, block " + blockId + " not found") } diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 2aa111d600e9b..98dcbf4e2dbfa 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -38,6 +38,7 @@ import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.executor.{DataReadMethod, InputMetrics} import org.apache.spark.util.NextIterator /** @@ -196,6 +197,20 @@ class HadoopRDD[K, V]( context.addOnCompleteCallback{ () => closeIfNeeded() } val key: K = reader.createKey() val value: V = reader.createValue() + + // Set the task input metrics. + val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) + try { + /* bytesRead may not exactly equal the bytes read by a task: split boundaries aren't + * always at record boundaries, so tasks may need to read into other splits to complete + * a record. */ + inputMetrics.bytesRead = split.inputSplit.value.getLength() + } catch { + case e: java.io.IOException => + logWarning("Unable to get input size to set InputMetrics for task", e) + } + context.taskMetrics.inputMetrics = Some(inputMetrics) + override def getNext() = { try { finished = !reader.next(key, value) diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index ac1ccc06f238a..f2b3a64bf1345 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -31,6 +31,7 @@ import org.apache.spark.Logging import org.apache.spark.Partition import org.apache.spark.SerializableWritable import org.apache.spark.{SparkContext, TaskContext} +import org.apache.spark.executor.{DataReadMethod, InputMetrics} private[spark] class NewHadoopPartition( rddId: Int, @@ -112,6 +113,18 @@ class NewHadoopRDD[K, V]( split.serializableHadoopSplit.value, hadoopAttemptContext) reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) + val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) + try { + /* bytesRead may not exactly equal the bytes read by a task: split boundaries aren't + * always at record boundaries, so tasks may need to read into other splits to complete + * a record. */ + inputMetrics.bytesRead = split.serializableHadoopSplit.value.getLength() + } catch { + case e: Exception => + logWarning("Unable to get input split size in order to set task input bytes", e) + } + context.taskMetrics.inputMetrics = Some(inputMetrics) + // Register an on-task-completion callback to close the input stream. context.addOnCompleteCallback(() => close()) var havePair = false diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index a1e21cad48b9b..47dd112f68325 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -26,7 +26,9 @@ import scala.collection.mutable.HashMap import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{DataReadMethod, TaskMetrics} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel /** * :: DeveloperApi :: @@ -160,7 +162,13 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener " START_TIME=" + taskInfo.launchTime + " FINISH_TIME=" + taskInfo.finishTime + " EXECUTOR_ID=" + taskInfo.executorId + " HOST=" + taskMetrics.hostname val executorRunTime = " EXECUTOR_RUN_TIME=" + taskMetrics.executorRunTime - val readMetrics = taskMetrics.shuffleReadMetrics match { + val inputMetrics = taskMetrics.inputMetrics match { + case Some(metrics) => + " READ_METHOD=" + metrics.readMethod.toString + + " INPUT_BYTES=" + metrics.bytesRead + case None => "" + } + val shuffleReadMetrics = taskMetrics.shuffleReadMetrics match { case Some(metrics) => " SHUFFLE_FINISH_TIME=" + metrics.shuffleFinishTime + " BLOCK_FETCHED_TOTAL=" + metrics.totalBlocksFetched + @@ -174,7 +182,8 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener case Some(metrics) => " SHUFFLE_BYTES_WRITTEN=" + metrics.shuffleBytesWritten case None => "" } - stageLogInfo(stageId, status + info + executorRunTime + readMetrics + writeMetrics) + stageLogInfo(stageId, status + info + executorRunTime + inputMetrics + shuffleReadMetrics + + writeMetrics) } /** 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 d2f7baf928b62..0db0a5bc7341b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -29,6 +29,7 @@ import akka.actor.{ActorSystem, Cancellable, Props} import sun.nio.ch.DirectBuffer import org.apache.spark._ +import org.apache.spark.executor.{DataReadMethod, InputMetrics} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer @@ -39,6 +40,15 @@ private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValu private[spark] case class IteratorValues(iterator: Iterator[Any]) extends BlockValues private[spark] case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends BlockValues +/* Class for returning a fetched block and associated metrics. */ +private[spark] class BlockResult( + val data: Iterator[Any], + readMethod: DataReadMethod.Value, + bytes: Long) { + val inputMetrics = new InputMetrics(readMethod) + inputMetrics.bytesRead = bytes +} + private[spark] class BlockManager( executorId: String, actorSystem: ActorSystem, @@ -334,9 +344,9 @@ private[spark] class BlockManager( /** * Get block from local block manager. */ - def getLocal(blockId: BlockId): Option[Iterator[Any]] = { + def getLocal(blockId: BlockId): Option[BlockResult] = { logDebug(s"Getting local block $blockId") - doGetLocal(blockId, asValues = true).asInstanceOf[Option[Iterator[Any]]] + doGetLocal(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] } /** @@ -355,11 +365,11 @@ private[spark] class BlockManager( blockId, s"Block $blockId not found on disk, though it should be") } } else { - doGetLocal(blockId, asValues = false).asInstanceOf[Option[ByteBuffer]] + doGetLocal(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] } } - private def doGetLocal(blockId: BlockId, asValues: Boolean): Option[Any] = { + private def doGetLocal(blockId: BlockId, asBlockResult: Boolean): Option[Any] = { val info = blockInfo.get(blockId).orNull if (info != null) { info.synchronized { @@ -386,14 +396,14 @@ private[spark] class BlockManager( // Look for the block in memory if (level.useMemory) { logDebug(s"Getting block $blockId from memory") - val result = if (asValues) { - memoryStore.getValues(blockId) + val result = if (asBlockResult) { + memoryStore.getValues(blockId).map(new BlockResult(_, DataReadMethod.Memory, info.size)) } else { memoryStore.getBytes(blockId) } result match { case Some(values) => - return Some(values) + return result case None => logDebug(s"Block $blockId not found in memory") } @@ -405,10 +415,11 @@ private[spark] class BlockManager( if (tachyonStore.contains(blockId)) { tachyonStore.getBytes(blockId) match { case Some(bytes) => - if (!asValues) { + if (!asBlockResult) { return Some(bytes) } else { - return Some(dataDeserialize(blockId, bytes)) + return Some(new BlockResult( + dataDeserialize(blockId, bytes), DataReadMethod.Memory, info.size)) } case None => logDebug(s"Block $blockId not found in tachyon") @@ -429,14 +440,15 @@ private[spark] class BlockManager( if (!level.useMemory) { // If the block shouldn't be stored in memory, we can just return it - if (asValues) { - return Some(dataDeserialize(blockId, bytes)) + if (asBlockResult) { + return Some(new BlockResult(dataDeserialize(blockId, bytes), DataReadMethod.Disk, + info.size)) } else { return Some(bytes) } } else { // Otherwise, we also have to store something in the memory store - if (!level.deserialized || !asValues) { + if (!level.deserialized || !asBlockResult) { /* We'll store the bytes in memory if the block's storage level includes * "memory serialized", or if it should be cached as objects in memory * but we only requested its serialized bytes. */ @@ -445,7 +457,7 @@ private[spark] class BlockManager( memoryStore.putBytes(blockId, copyForMemory, level) bytes.rewind() } - if (!asValues) { + if (!asBlockResult) { return Some(bytes) } else { val values = dataDeserialize(blockId, bytes) @@ -457,12 +469,12 @@ private[spark] class BlockManager( memoryStore.putValues(blockId, valuesBuffer, level, returnValues = true).data match { case Left(values2) => - return Some(values2) + return Some(new BlockResult(values2, DataReadMethod.Disk, info.size)) case _ => - throw new SparkException("Memory store did not return an iterator") + throw new SparkException("Memory store did not return back an iterator") } } else { - return Some(values) + return Some(new BlockResult(values, DataReadMethod.Disk, info.size)) } } } @@ -477,9 +489,9 @@ private[spark] class BlockManager( /** * Get block from remote block managers. */ - def getRemote(blockId: BlockId): Option[Iterator[Any]] = { + def getRemote(blockId: BlockId): Option[BlockResult] = { logDebug(s"Getting remote block $blockId") - doGetRemote(blockId, asValues = true).asInstanceOf[Option[Iterator[Any]]] + doGetRemote(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] } /** @@ -487,10 +499,10 @@ private[spark] class BlockManager( */ def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = { logDebug(s"Getting remote block $blockId as bytes") - doGetRemote(blockId, asValues = false).asInstanceOf[Option[ByteBuffer]] + doGetRemote(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] } - private def doGetRemote(blockId: BlockId, asValues: Boolean): Option[Any] = { + private def doGetRemote(blockId: BlockId, asBlockResult: Boolean): Option[Any] = { require(blockId != null, "BlockId is null") val locations = Random.shuffle(master.getLocations(blockId)) for (loc <- locations) { @@ -498,8 +510,11 @@ private[spark] class BlockManager( val data = BlockManagerWorker.syncGetBlock( GetBlock(blockId), ConnectionManagerId(loc.host, loc.port)) if (data != null) { - if (asValues) { - return Some(dataDeserialize(blockId, data)) + if (asBlockResult) { + return Some(new BlockResult( + dataDeserialize(blockId, data), + DataReadMethod.Network, + data.limit())) } else { return Some(data) } @@ -513,7 +528,7 @@ private[spark] class BlockManager( /** * Get a block from the block manager (either local or remote). */ - def get(blockId: BlockId): Option[Iterator[Any]] = { + def get(blockId: BlockId): Option[BlockResult] = { val local = getLocal(blockId) if (local.isDefined) { logInfo(s"Found block $blockId locally") @@ -792,7 +807,7 @@ private[spark] class BlockManager( * Read a block consisting of a single object. */ def getSingle(blockId: BlockId): Option[Any] = { - get(blockId).map(_.next()) + get(blockId).map(_.data.next()) } /** 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 a107c5182b3be..328be158db680 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -78,7 +78,7 @@ private[spark] object ThreadingTest { val startTime = System.currentTimeMillis() manager.get(blockId) match { case Some(retrievedBlock) => - assert(retrievedBlock.toList.asInstanceOf[List[Int]] == block.toList, + assert(retrievedBlock.data.toList.asInstanceOf[List[Int]] == block.toList, "Block " + blockId + " did not match") println("Got block " + blockId + " in " + (System.currentTimeMillis - startTime) + " ms") diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 6cfc46c7e78fe..9625337ae21a5 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -72,6 +72,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { "Complete Tasks", "Total Tasks", "Task Time", + "Input Bytes", "Shuffle Read", "Shuffle Write") @@ -97,6 +98,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { + @@ -119,6 +121,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { val completedTasks = listener.executorToTasksComplete.getOrElse(execId, 0) val totalTasks = activeTasks + failedTasks + completedTasks val totalDuration = listener.executorToDuration.getOrElse(execId, 0) + val totalInputBytes = listener.executorToInputBytes.getOrElse(execId, 0) val totalShuffleRead = listener.executorToShuffleRead.getOrElse(execId, 0) val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0) @@ -136,6 +139,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { completedTasks, totalTasks, totalDuration, + totalInputBytes, totalShuffleRead, totalShuffleWrite, maxMem diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 91d37b835b19d..58eeb86bf9a3a 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -46,6 +46,7 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() val executorToDuration = HashMap[String, Long]() + val executorToInputBytes = HashMap[String, Long]() val executorToShuffleRead = HashMap[String, Long]() val executorToShuffleWrite = HashMap[String, Long]() @@ -72,6 +73,10 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) // Update shuffle read/write val metrics = taskEnd.taskMetrics if (metrics != null) { + metrics.inputMetrics.foreach { inputMetrics => + executorToInputBytes(eid) = + executorToInputBytes.getOrElse(eid, 0L) + inputMetrics.bytesRead + } metrics.shuffleReadMetrics.foreach { shuffleRead => executorToShuffleRead(eid) = executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala index 2aaf6329b792d..c4a8996c0b9a9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala @@ -28,6 +28,7 @@ class ExecutorSummary { var taskTime : Long = 0 var failedTasks : Int = 0 var succeededTasks : Int = 0 + var inputBytes: Long = 0 var shuffleRead : Long = 0 var shuffleWrite : Long = 0 var memoryBytesSpilled : Long = 0 diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index add0e9878a546..2a34a9af925d6 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -43,6 +43,7 @@ private[ui] class ExecutorTable(stageId: Int, parent: JobProgressTab) { + @@ -75,6 +76,7 @@ private[ui] class ExecutorTable(stageId: Int, parent: JobProgressTab) { + diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 381a5443df8b5..2286a7f952f28 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -46,13 +46,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener { val completedStages = ListBuffer[StageInfo]() val failedStages = ListBuffer[StageInfo]() - // Total metrics reflect metrics only for completed tasks - var totalTime = 0L - var totalShuffleRead = 0L - var totalShuffleWrite = 0L - // TODO: Should probably consolidate all following into a single hash map. val stageIdToTime = HashMap[Int, Long]() + val stageIdToInputBytes = HashMap[Int, Long]() val stageIdToShuffleRead = HashMap[Int, Long]() val stageIdToShuffleWrite = HashMap[Int, Long]() val stageIdToMemoryBytesSpilled = HashMap[Int, Long]() @@ -93,6 +89,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener { val toRemove = math.max(retainedStages / 10, 1) stages.take(toRemove).foreach { s => stageIdToTime.remove(s.stageId) + stageIdToInputBytes.remove(s.stageId) stageIdToShuffleRead.remove(s.stageId) stageIdToShuffleWrite.remove(s.stageId) stageIdToMemoryBytesSpilled.remove(s.stageId) @@ -171,6 +168,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener { val metrics = taskEnd.taskMetrics if (metrics != null) { + metrics.inputMetrics.foreach { y.inputBytes += _.bytesRead } metrics.shuffleReadMetrics.foreach { y.shuffleRead += _.remoteBytesRead } metrics.shuffleWriteMetrics.foreach { y.shuffleWrite += _.shuffleBytesWritten } y.memoryBytesSpilled += metrics.memoryBytesSpilled @@ -200,18 +198,19 @@ class JobProgressListener(conf: SparkConf) extends SparkListener { stageIdToTime.getOrElseUpdate(sid, 0L) val time = metrics.map(_.executorRunTime).getOrElse(0L) stageIdToTime(sid) += time - totalTime += time + + stageIdToInputBytes.getOrElseUpdate(sid, 0L) + val inputBytes = metrics.flatMap(_.inputMetrics).map(_.bytesRead).getOrElse(0L) + stageIdToInputBytes(sid) += inputBytes stageIdToShuffleRead.getOrElseUpdate(sid, 0L) val shuffleRead = metrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead).getOrElse(0L) stageIdToShuffleRead(sid) += shuffleRead - totalShuffleRead += shuffleRead stageIdToShuffleWrite.getOrElseUpdate(sid, 0L) val shuffleWrite = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten).getOrElse(0L) stageIdToShuffleWrite(sid) += shuffleWrite - totalShuffleWrite += shuffleWrite stageIdToMemoryBytesSpilled.getOrElseUpdate(sid, 0L) val memoryBytesSpilled = metrics.map(_.memoryBytesSpilled).getOrElse(0L) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 8e3d5d1cd4c6b..afb8ed754ff8b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -48,6 +48,8 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val tasks = listener.stageIdToTaskData(stageId).values.toSeq.sortBy(_.taskInfo.launchTime) val numCompleted = tasks.count(_.taskInfo.finished) + val inputBytes = listener.stageIdToInputBytes.getOrElse(stageId, 0L) + val hasInput = inputBytes > 0 val shuffleReadBytes = listener.stageIdToShuffleRead.getOrElse(stageId, 0L) val hasShuffleRead = shuffleReadBytes > 0 val shuffleWriteBytes = listener.stageIdToShuffleWrite.getOrElse(stageId, 0L) @@ -69,6 +71,12 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { Total task time across all tasks: {UIUtils.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)} + {if (hasInput) +
  • + Input: + {Utils.bytesToString(inputBytes)} +
  • + } {if (hasShuffleRead)
  • Shuffle read: @@ -98,13 +106,14 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { Seq( "Index", "ID", "Attempt", "Status", "Locality Level", "Executor", "Launch Time", "Duration", "GC Time") ++ + {if (hasInput) Seq("Input") else Nil} ++ {if (hasShuffleRead) Seq("Shuffle Read") else Nil} ++ {if (hasShuffleWrite) Seq("Write Time", "Shuffle Write") else Nil} ++ {if (hasBytesSpilled) Seq("Shuffle Spill (Memory)", "Shuffle Spill (Disk)") else Nil} ++ Seq("Errors") val taskTable = UIUtils.listingTable( - taskHeaders, taskRow(hasShuffleRead, hasShuffleWrite, hasBytesSpilled), tasks) + taskHeaders, taskRow(hasInput, hasShuffleRead, hasShuffleWrite, hasBytesSpilled), tasks) // Excludes tasks which failed and have incomplete metrics val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.taskMetrics.isDefined) @@ -159,6 +168,11 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { def getQuantileCols(data: Seq[Double]) = Distribution(data).get.getQuantiles().map(d => Utils.bytesToString(d.toLong)) + val inputSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.inputMetrics.map(_.bytesRead).getOrElse(0L).toDouble + } + val inputQuantiles = "Input" +: getQuantileCols(inputSizes) + val shuffleReadSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble } @@ -186,6 +200,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { serviceQuantiles, gettingResultQuantiles, schedulerDelayQuantiles, + if (hasInput) inputQuantiles else Nil, if (hasShuffleRead) shuffleReadQuantiles else Nil, if (hasShuffleWrite) shuffleWriteQuantiles else Nil, if (hasBytesSpilled) memoryBytesSpilledQuantiles else Nil, @@ -209,8 +224,11 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { } } - def taskRow(shuffleRead: Boolean, shuffleWrite: Boolean, bytesSpilled: Boolean) - (taskData: TaskUIData): Seq[Node] = { + def taskRow( + hasInput: Boolean, + hasShuffleRead: Boolean, + hasShuffleWrite: Boolean, + hasBytesSpilled: Boolean)(taskData: TaskUIData): Seq[Node] = { taskData match { case TaskUIData(info, metrics, errorMessage) => val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) else metrics.map(_.executorRunTime).getOrElse(1L) @@ -219,6 +237,12 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) + val maybeInput = metrics.flatMap(_.inputMetrics) + val inputSortable = maybeInput.map(_.bytesRead.toString).getOrElse("") + val inputReadable = maybeInput + .map(m => s"${Utils.bytesToString(m.bytesRead)} (${m.readMethod.toString.toLowerCase()})") + .getOrElse("") + val maybeShuffleRead = metrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead) val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") val shuffleReadReadable = maybeShuffleRead.map(Utils.bytesToString).getOrElse("") @@ -265,12 +289,17 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { {if (serializationTime > 0) UIUtils.formatDuration(serializationTime) else ""} --> - {if (shuffleRead) { + {if (hasInput) { +
  • + }} + {if (hasShuffleRead) { }} - {if (shuffleWrite) { + {if (hasShuffleWrite) { @@ -278,7 +307,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { {shuffleWriteReadable} }} - {if (bytesSpilled) { + {if (hasBytesSpilled) { 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 b2b6cc6a6ec2d..a9ac6d5bee9c9 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 @@ -43,6 +43,7 @@ private[ui] class StageTableBase( + } @@ -123,6 +124,11 @@ private[ui] class StageTableBase( case _ => "" } val totalTasks = s.numTasks + val inputSortable = listener.stageIdToInputBytes.getOrElse(s.stageId, 0L) + val inputRead = inputSortable match { + case 0 => "" + case b => Utils.bytesToString(b) + } val shuffleReadSortable = listener.stageIdToShuffleRead.getOrElse(s.stageId, 0L) val shuffleRead = shuffleReadSortable match { case 0 => "" @@ -150,6 +156,7 @@ private[ui] class StageTableBase( + } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 6245b4b8023c2..26c9c9d6034ec 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -26,7 +26,8 @@ import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.json4s.JsonAST._ -import org.apache.spark.executor.{ShuffleReadMetrics, ShuffleWriteMetrics, TaskMetrics} +import org.apache.spark.executor.{DataReadMethod, InputMetrics, ShuffleReadMetrics, + ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.scheduler._ import org.apache.spark.storage._ import org.apache.spark._ @@ -213,6 +214,8 @@ private[spark] object JsonProtocol { taskMetrics.shuffleReadMetrics.map(shuffleReadMetricsToJson).getOrElse(JNothing) val shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics.map(shuffleWriteMetricsToJson).getOrElse(JNothing) + val inputMetrics = + taskMetrics.inputMetrics.map(inputMetricsToJson).getOrElse(JNothing) val updatedBlocks = taskMetrics.updatedBlocks.map { blocks => JArray(blocks.toList.map { case (id, status) => @@ -230,6 +233,7 @@ private[spark] object JsonProtocol { ("Disk Bytes Spilled" -> taskMetrics.diskBytesSpilled) ~ ("Shuffle Read Metrics" -> shuffleReadMetrics) ~ ("Shuffle Write Metrics" -> shuffleWriteMetrics) ~ + ("Input Metrics" -> inputMetrics) ~ ("Updated Blocks" -> updatedBlocks) } @@ -247,6 +251,11 @@ private[spark] object JsonProtocol { ("Shuffle Write Time" -> shuffleWriteMetrics.shuffleWriteTime) } + def inputMetricsToJson(inputMetrics: InputMetrics): JValue = { + ("Data Read Method" -> inputMetrics.readMethod.toString) ~ + ("Bytes Read" -> inputMetrics.bytesRead) + } + def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { val reason = Utils.getFormattedClassName(taskEndReason) val json = taskEndReason match { @@ -528,6 +537,8 @@ private[spark] object JsonProtocol { Utils.jsonOption(json \ "Shuffle Read Metrics").map(shuffleReadMetricsFromJson) metrics.shuffleWriteMetrics = Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) + metrics.inputMetrics = + Utils.jsonOption(json \ "Input Metrics").map(inputMetricsFromJson) metrics.updatedBlocks = Utils.jsonOption(json \ "Updated Blocks").map { value => value.extract[List[JValue]].map { block => @@ -557,6 +568,13 @@ private[spark] object JsonProtocol { metrics } + def inputMetricsFromJson(json: JValue): InputMetrics = { + val metrics = new InputMetrics( + DataReadMethod.withName((json \ "Data Read Method").extract[String])) + metrics.bytesRead = (json \ "Bytes Read").extract[Long] + metrics + } + def taskEndReasonFromJson(json: JValue): TaskEndReason = { val success = Utils.getFormattedClassName(Success) val resubmitted = Utils.getFormattedClassName(Resubmitted) diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index 4f178db40f638..7f5d0b061e8b0 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -22,6 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.mock.EasyMockSugar +import org.apache.spark.executor.{DataReadMethod, TaskMetrics} import org.apache.spark.rdd.RDD import org.apache.spark.storage._ @@ -66,7 +67,8 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar test("get cached rdd") { expecting { - blockManager.get(RDDBlockId(0, 0)).andReturn(Some(ArrayBuffer(5, 6, 7).iterator)) + val result = new BlockResult(ArrayBuffer(5, 6, 7).iterator, DataReadMethod.Memory, 12) + blockManager.get(RDDBlockId(0, 0)).andReturn(Some(result)) } whenExecuting(blockManager) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 6df0a080961b6..71f48e295ecca 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -251,6 +251,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers taskInfoMetrics.foreach { case (taskInfo, taskMetrics) => taskMetrics.resultSize should be > (0l) if (stageInfo.rddInfos.exists(info => info.name == d2.name || info.name == d3.name)) { + taskMetrics.inputMetrics should not be ('defined) taskMetrics.shuffleWriteMetrics should be ('defined) taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0l) } 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 d7dbe5164b7f6..23cb6905bfdeb 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -31,11 +31,13 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers import org.scalatest.time.SpanSugar._ -import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} +import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf, SparkContext} +import org.apache.spark.executor.DataReadMethod import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} +import scala.collection.mutable.ArrayBuffer import scala.language.implicitConversions import scala.language.postfixOps @@ -415,6 +417,39 @@ 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) + val list1Get = store.get("list1") + assert(list1Get.isDefined, "list1 expected to be in store") + assert(list1Get.get.data.size === 2) + assert(list1Get.get.inputMetrics.bytesRead === list1SizeEstimate) + assert(list1Get.get.inputMetrics.readMethod === DataReadMethod.Memory) + val list2MemoryGet = store.get("list2memory") + assert(list2MemoryGet.isDefined, "list2memory expected to be in store") + assert(list2MemoryGet.get.data.size === 3) + assert(list2MemoryGet.get.inputMetrics.bytesRead === list2SizeEstimate) + assert(list2MemoryGet.get.inputMetrics.readMethod === DataReadMethod.Memory) + val list2DiskGet = store.get("list2disk") + assert(list2DiskGet.isDefined, "list2memory expected to be in store") + assert(list2DiskGet.get.data.size === 3) + System.out.println(list2DiskGet) + // We don't know the exact size of the data on disk, but it should certainly be > 0. + assert(list2DiskGet.get.inputMetrics.bytesRead > 0) + assert(list2DiskGet.get.inputMetrics.readMethod === DataReadMethod.Disk) + } + test("in-memory LRU storage") { store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr, mapOutputTracker) @@ -630,18 +665,18 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) store.put("list3", list3.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.get("list2").isDefined, "list2 was not in store") - assert(store.get("list2").get.size == 2) + assert(store.get("list2").get.data.size === 2) assert(store.get("list3").isDefined, "list3 was not in store") - assert(store.get("list3").get.size == 2) + assert(store.get("list3").get.data.size === 2) assert(store.get("list1") === None, "list1 was in store") assert(store.get("list2").isDefined, "list2 was not in store") - assert(store.get("list2").get.size == 2) + 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) assert(store.get("list1").isDefined, "list1 was not in store") - assert(store.get("list1").get.size == 2) + assert(store.get("list1").get.data.size === 2) assert(store.get("list2").isDefined, "list2 was not in store") - assert(store.get("list2").get.size == 2) + assert(store.get("list2").get.data.size === 2) assert(store.get("list3") === None, "list1 was in store") } @@ -656,28 +691,31 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter 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) + val listForSizeEstimate = new ArrayBuffer[Any] + listForSizeEstimate ++= list1.iterator + val listSize = SizeEstimator.estimate(listForSizeEstimate) // At this point LRU should not kick in because list3 is only on disk - assert(store.get("list1").isDefined, "list2 was not in store") - assert(store.get("list1").get.size === 2) - assert(store.get("list2").isDefined, "list3 was not in store") - assert(store.get("list2").get.size === 2) - assert(store.get("list3").isDefined, "list1 was not in store") - assert(store.get("list3").get.size === 2) - assert(store.get("list1").isDefined, "list2 was not in store") - assert(store.get("list1").get.size === 2) - assert(store.get("list2").isDefined, "list3 was not in store") - assert(store.get("list2").get.size === 2) - assert(store.get("list3").isDefined, "list1 was not in store") - assert(store.get("list3").get.size === 2) + 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") + assert(store.get("list2").get.data.size === 2) + assert(store.get("list3").isDefined, "list3 was not in store") + assert(store.get("list3").get.data.size === 2) + 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") + assert(store.get("list2").get.data.size === 2) + 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) assert(store.get("list1") === None, "list1 was in store") - assert(store.get("list2").isDefined, "list3 was not in store") - assert(store.get("list2").get.size === 2) - assert(store.get("list3").isDefined, "list1 was not in store") - assert(store.get("list3").get.size === 2) + 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") + assert(store.get("list3").get.data.size === 2) assert(store.get("list4").isDefined, "list4 was not in store") - assert(store.get("list4").get.size === 2) + assert(store.get("list4").get.data.size === 2) } test("negative byte values in ByteBufferInputStream") { diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 6c49870455873..316e14100e409 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -39,7 +39,11 @@ class JsonProtocolSuite extends FunSuite { val taskGettingResult = SparkListenerTaskGettingResult(makeTaskInfo(1000L, 2000, 5, 3000L, true)) val taskEnd = SparkListenerTaskEnd(1, "ShuffleMapTask", Success, - makeTaskInfo(123L, 234, 67, 345L, false), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800)) + makeTaskInfo(123L, 234, 67, 345L, false), + makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = false)) + val taskEndWithHadoopInput = SparkListenerTaskEnd(1, "ShuffleMapTask", Success, + makeTaskInfo(123L, 234, 67, 345L, false), + makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true)) val jobStart = SparkListenerJobStart(10, Seq[Int](1, 2, 3, 4), properties) val jobEnd = SparkListenerJobEnd(20, JobSucceeded) val environmentUpdate = SparkListenerEnvironmentUpdate(Map[String, Seq[(String, String)]]( @@ -61,6 +65,7 @@ class JsonProtocolSuite extends FunSuite { testEvent(taskStart, taskStartJsonString) testEvent(taskGettingResult, taskGettingResultJsonString) testEvent(taskEnd, taskEndJsonString) + testEvent(taskEndWithHadoopInput, taskEndWithHadoopInputJsonString) testEvent(jobStart, jobStartJsonString) testEvent(jobEnd, jobEndJsonString) testEvent(environmentUpdate, environmentUpdateJsonString) @@ -75,7 +80,7 @@ class JsonProtocolSuite extends FunSuite { testRDDInfo(makeRddInfo(2, 3, 4, 5L, 6L)) testStageInfo(makeStageInfo(10, 20, 30, 40L, 50L)) testTaskInfo(makeTaskInfo(999L, 888, 55, 777L, false)) - testTaskMetrics(makeTaskMetrics(33333L, 44444L, 55555L, 66666L, 7, 8)) + testTaskMetrics(makeTaskMetrics(33333L, 44444L, 55555L, 66666L, 7, 8, hasHadoopInput = false)) testBlockManagerId(BlockManagerId("Hong", "Kong", 500, 1000)) // StorageLevel @@ -118,7 +123,7 @@ class JsonProtocolSuite extends FunSuite { testBlockId(StreamBlockId(1, 2L)) } - test("Backward compatibility") { + test("StageInfo.details backward compatibility") { // StageInfo.details was added after 1.0.0. val info = makeStageInfo(1, 2, 3, 4L, 5L) assert(info.details.nonEmpty) @@ -129,6 +134,16 @@ class JsonProtocolSuite extends FunSuite { assert("" === newInfo.details) } + test("InputMetrics backward compatibility") { + // InputMetrics were added after 1.0.1. + val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6, hasHadoopInput = true) + assert(metrics.inputMetrics.nonEmpty) + val newJson = JsonProtocol.taskMetricsToJson(metrics) + val oldJson = newJson.removeField { case (field, _) => field == "Input Metrics" } + val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson) + assert(newMetrics.inputMetrics.isEmpty) + } + /** -------------------------- * | Helper test running methods | @@ -294,6 +309,8 @@ class JsonProtocolSuite extends FunSuite { metrics1.shuffleReadMetrics, metrics2.shuffleReadMetrics, assertShuffleReadEquals) assertOptionEquals( metrics1.shuffleWriteMetrics, metrics2.shuffleWriteMetrics, assertShuffleWriteEquals) + assertOptionEquals( + metrics1.inputMetrics, metrics2.inputMetrics, assertInputMetricsEquals) assertOptionEquals(metrics1.updatedBlocks, metrics2.updatedBlocks, assertBlocksEquals) } @@ -311,6 +328,11 @@ class JsonProtocolSuite extends FunSuite { assert(metrics1.shuffleWriteTime === metrics2.shuffleWriteTime) } + private def assertEquals(metrics1: InputMetrics, metrics2: InputMetrics) { + assert(metrics1.readMethod === metrics2.readMethod) + assert(metrics1.bytesRead === metrics2.bytesRead) + } + private def assertEquals(bm1: BlockManagerId, bm2: BlockManagerId) { assert(bm1.executorId === bm2.executorId) assert(bm1.host === bm2.host) @@ -403,6 +425,10 @@ class JsonProtocolSuite extends FunSuite { assertEquals(w1, w2) } + private def assertInputMetricsEquals(i1: InputMetrics, i2: InputMetrics) { + assertEquals(i1, i2) + } + private def assertTaskMetricsEquals(t1: TaskMetrics, t2: TaskMetrics) { assertEquals(t1, t2) } @@ -460,9 +486,19 @@ class JsonProtocolSuite extends FunSuite { new TaskInfo(a, b, c, d, "executor", "your kind sir", TaskLocality.NODE_LOCAL, speculative) } - private def makeTaskMetrics(a: Long, b: Long, c: Long, d: Long, e: Int, f: Int) = { + /** + * Creates a TaskMetrics object describing a task that read data from Hadoop (if hasHadoopInput is + * set to true) or read data from a shuffle otherwise. + */ + private def makeTaskMetrics( + a: Long, + b: Long, + c: Long, + d: Long, + e: Int, + f: Int, + hasHadoopInput: Boolean) = { val t = new TaskMetrics - val sr = new ShuffleReadMetrics val sw = new ShuffleWriteMetrics t.hostname = "localhost" t.executorDeserializeTime = a @@ -471,15 +507,23 @@ class JsonProtocolSuite extends FunSuite { t.jvmGCTime = d t.resultSerializationTime = a + b t.memoryBytesSpilled = a + c - sr.shuffleFinishTime = b + c - sr.totalBlocksFetched = e + f - sr.remoteBytesRead = b + d - sr.localBlocksFetched = e - sr.fetchWaitTime = a + d - sr.remoteBlocksFetched = f + + if (hasHadoopInput) { + val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) + inputMetrics.bytesRead = d + e + f + t.inputMetrics = Some(inputMetrics) + } else { + val sr = new ShuffleReadMetrics + sr.shuffleFinishTime = b + c + sr.totalBlocksFetched = e + f + sr.remoteBytesRead = b + d + sr.localBlocksFetched = e + sr.fetchWaitTime = a + d + sr.remoteBlocksFetched = f + t.shuffleReadMetrics = Some(sr) + } sw.shuffleBytesWritten = a + b + c sw.shuffleWriteTime = b + c + d - t.shuffleReadMetrics = Some(sr) t.shuffleWriteMetrics = Some(sw) // Make at most 6 blocks t.updatedBlocks = Some((1 to (e % 5 + 1)).map { i => @@ -552,8 +596,9 @@ class JsonProtocolSuite extends FunSuite { | }, | "Shuffle Write Metrics":{ | "Shuffle Bytes Written":1200, - | "Shuffle Write Time":1500}, - | "Updated Blocks":[ + | "Shuffle Write Time":1500 + | }, + | "Updated Blocks":[ | {"Block ID":"rdd_0_0", | "Status":{ | "Storage Level":{ @@ -568,6 +613,35 @@ class JsonProtocolSuite extends FunSuite { |} """.stripMargin + private val taskEndWithHadoopInputJsonString = + """ + |{"Event":"SparkListenerTaskEnd","Stage ID":1,"Task Type":"ShuffleMapTask", + |"Task End Reason":{"Reason":"Success"}, + |"Task Info":{ + | "Task ID":123,"Index":234,"Attempt":67,"Launch Time":345,"Executor ID":"executor", + | "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":false, + | "Getting Result Time":0,"Finish Time":0,"Failed":false,"Serialized Size":0 + |}, + |"Task Metrics":{ + | "Host Name":"localhost","Executor Deserialize Time":300,"Executor Run Time":400, + | "Result Size":500,"JVM GC Time":600,"Result Serialization Time":700, + | "Memory Bytes Spilled":800,"Disk Bytes Spilled":0, + | "Shuffle Write Metrics":{"Shuffle Bytes Written":1200,"Shuffle Write Time":1500}, + | "Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":2100}, + | "Updated Blocks":[ + | {"Block ID":"rdd_0_0", + | "Status":{ + | "Storage Level":{ + | "Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":false, + | "Replication":2 + | }, + | "Memory Size":0,"Tachyon Size":0,"Disk Size":0 + | } + | } + | ]} + |} + """ + private val jobStartJsonString = """ {"Event":"SparkListenerJobStart","Job ID":10,"Stage IDs":[1,2,3,4],"Properties": From 66135a341d9f8baecc149d13ae5511f14578c395 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Sun, 29 Jun 2014 23:00:00 -0700 Subject: [PATCH 072/124] [SPARK-2104] Fix task serializing issues when sort with Java non serializable class Details can be see in [SPARK-2104](https://issues.apache.org/jira/browse/SPARK-2104). This work is based on Reynold's work, add some unit tests to validate the issue. @rxin , would you please take a look at this PR, thanks a lot. Author: jerryshao Closes #1245 from jerryshao/SPARK-2104 and squashes the following commits: c8ee362 [jerryshao] Make field partitions transient 2b41917 [jerryshao] Minor changes 47d763c [jerryshao] Fix task serializing issue when sort with Java non serializable class --- .../scala/org/apache/spark/Partitioner.scala | 52 ++++++++++++++++--- .../scala/org/apache/spark/ShuffleSuite.scala | 42 ++++++++++++++- 2 files changed, 86 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index e7f75481939a8..ec99648a8488a 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -17,11 +17,13 @@ package org.apache.spark +import java.io.{IOException, ObjectInputStream, ObjectOutputStream} + import scala.reflect.ClassTag import org.apache.spark.rdd.RDD -import org.apache.spark.util.CollectionsUtils -import org.apache.spark.util.Utils +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.util.{CollectionsUtils, Utils} /** * An object that defines how the elements in a key-value pair RDD are partitioned by key. @@ -96,15 +98,15 @@ class HashPartitioner(partitions: Int) extends Partitioner { * the value of `partitions`. */ class RangePartitioner[K : Ordering : ClassTag, V]( - partitions: Int, + @transient partitions: Int, @transient rdd: RDD[_ <: Product2[K,V]], - private val ascending: Boolean = true) + private var ascending: Boolean = true) extends Partitioner { - private val ordering = implicitly[Ordering[K]] + private var ordering = implicitly[Ordering[K]] // An array of upper bounds for the first (partitions - 1) partitions - private val rangeBounds: Array[K] = { + private var rangeBounds: Array[K] = { if (partitions == 1) { Array() } else { @@ -127,7 +129,7 @@ class RangePartitioner[K : Ordering : ClassTag, V]( def numPartitions = rangeBounds.length + 1 - private val binarySearch: ((Array[K], K) => Int) = CollectionsUtils.makeBinarySearch[K] + private var binarySearch: ((Array[K], K) => Int) = CollectionsUtils.makeBinarySearch[K] def getPartition(key: Any): Int = { val k = key.asInstanceOf[K] @@ -173,4 +175,40 @@ class RangePartitioner[K : Ordering : ClassTag, V]( result = prime * result + ascending.hashCode result } + + @throws(classOf[IOException]) + private def writeObject(out: ObjectOutputStream) { + val sfactory = SparkEnv.get.serializer + sfactory match { + case js: JavaSerializer => out.defaultWriteObject() + case _ => + out.writeBoolean(ascending) + out.writeObject(ordering) + out.writeObject(binarySearch) + + val ser = sfactory.newInstance() + Utils.serializeViaNestedStream(out, ser) { stream => + stream.writeObject(scala.reflect.classTag[Array[K]]) + stream.writeObject(rangeBounds) + } + } + } + + @throws(classOf[IOException]) + private def readObject(in: ObjectInputStream) { + val sfactory = SparkEnv.get.serializer + sfactory match { + case js: JavaSerializer => in.defaultReadObject() + case _ => + ascending = in.readBoolean() + ordering = in.readObject().asInstanceOf[Ordering[K]] + binarySearch = in.readObject().asInstanceOf[(Array[K], K) => Int] + + val ser = sfactory.newInstance() + Utils.deserializeViaNestedStream(in, ser) { ds => + implicit val classTag = ds.readObject[ClassTag[Array[K]]]() + rangeBounds = ds.readObject[Array[K]]() + } + } + } } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index b40fee7e9ab23..c4f2f7e34f4d5 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -206,6 +206,42 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { // substracted rdd return results as Tuple2 results(0) should be ((3, 33)) } + + test("sort with Java non serializable class - Kryo") { + // Use a local cluster with 2 processes to make sure there are both local and remote blocks + val conf = new SparkConf() + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .setAppName("test") + .setMaster("local-cluster[2,1,512]") + sc = new SparkContext(conf) + val a = sc.parallelize(1 to 10, 2) + val b = a.map { x => + (new NonJavaSerializableClass(x), x) + } + // If the Kryo serializer is not used correctly, the shuffle would fail because the + // default Java serializer cannot handle the non serializable class. + val c = b.sortByKey().map(x => x._2) + assert(c.collect() === Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)) + } + + test("sort with Java non serializable class - Java") { + // Use a local cluster with 2 processes to make sure there are both local and remote blocks + val conf = new SparkConf() + .setAppName("test") + .setMaster("local-cluster[2,1,512]") + sc = new SparkContext(conf) + val a = sc.parallelize(1 to 10, 2) + val b = a.map { x => + (new NonJavaSerializableClass(x), x) + } + // default Java serializer cannot handle the non serializable class. + val thrown = intercept[SparkException] { + b.sortByKey().collect() + } + + assert(thrown.getClass === classOf[SparkException]) + assert(thrown.getMessage.contains("NotSerializableException")) + } } object ShuffleSuite { @@ -215,5 +251,9 @@ object ShuffleSuite { x + y } - class NonJavaSerializableClass(val value: Int) + class NonJavaSerializableClass(val value: Int) extends Comparable[NonJavaSerializableClass] { + override def compareTo(o: NonJavaSerializableClass): Int = { + value - o.value + } + } } From a484030dae9d0d7e4b97cc6307e9e928c07490dc Mon Sep 17 00:00:00 2001 From: William Benton Date: Sun, 29 Jun 2014 23:27:34 -0700 Subject: [PATCH 073/124] SPARK-897: preemptively serialize closures These commits cause `ClosureCleaner.clean` to attempt to serialize the cleaned closure with the default closure serializer and throw a `SparkException` if doing so fails. This behavior is enabled by default but can be disabled at individual callsites of `SparkContext.clean`. Commit 98e01ae8 fixes some no-op assertions in `GraphSuite` that this work exposed; I'm happy to put that in a separate PR if that would be more appropriate. Author: William Benton Closes #143 from willb/spark-897 and squashes the following commits: bceab8a [William Benton] Commented DStream corner cases for serializability checking. 64d04d2 [William Benton] FailureSuite now checks both messages and causes. 3b3f74a [William Benton] Stylistic and doc cleanups. b215dea [William Benton] Fixed spurious failures in ImplicitOrderingSuite be1ecd6 [William Benton] Don't check serializability of DStream transforms. abe816b [William Benton] Make proactive serializability checking optional. 5bfff24 [William Benton] Adds proactive closure-serializablilty checking ed2ccf0 [William Benton] Test cases for SPARK-897. --- .../scala/org/apache/spark/SparkContext.scala | 12 ++- .../apache/spark/util/ClosureCleaner.scala | 16 +++- .../scala/org/apache/spark/FailureSuite.scala | 14 ++- .../apache/spark/ImplicitOrderingSuite.scala | 75 +++++++++++----- .../ProactiveClosureSerializationSuite.scala | 90 +++++++++++++++++++ .../spark/streaming/dstream/DStream.scala | 25 ++++-- 6 files changed, 196 insertions(+), 36 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index f9476ff826a62..8819e73d17fb2 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1203,9 +1203,17 @@ class SparkContext(config: SparkConf) extends Logging { /** * Clean a closure to make it ready to serialized and send to tasks * (removes unreferenced variables in $outer's, updates REPL variables) + * If checkSerializable is set, clean will also proactively + * check to see if f is serializable and throw a SparkException + * if not. + * + * @param f the closure to clean + * @param checkSerializable whether or not to immediately check f for serializability + * @throws SparkException if checkSerializable is set but f is not + * serializable */ - private[spark] def clean[F <: AnyRef](f: F): F = { - ClosureCleaner.clean(f) + private[spark] def clean[F <: AnyRef](f: F, checkSerializable: Boolean = true): F = { + ClosureCleaner.clean(f, checkSerializable) f } diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index 4916d9b86cca5..e3f52f6ff1e63 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.Set import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ -import org.apache.spark.{Logging, SparkException} +import org.apache.spark.{Logging, SparkEnv, SparkException} private[spark] object ClosureCleaner extends Logging { // Get an ASM class reader for a given class from the JAR that loaded it @@ -101,7 +101,7 @@ private[spark] object ClosureCleaner extends Logging { } } - def clean(func: AnyRef) { + def clean(func: AnyRef, checkSerializable: Boolean = true) { // TODO: cache outerClasses / innerClasses / accessedFields val outerClasses = getOuterClasses(func) val innerClasses = getInnerClasses(func) @@ -153,6 +153,18 @@ private[spark] object ClosureCleaner extends Logging { field.setAccessible(true) field.set(func, outer) } + + if (checkSerializable) { + ensureSerializable(func) + } + } + + private def ensureSerializable(func: AnyRef) { + try { + SparkEnv.get.closureSerializer.newInstance().serialize(func) + } catch { + case ex: Exception => throw new SparkException("Task not serializable", ex) + } } private def instantiateClass(cls: Class[_], outer: AnyRef, inInterpreter: Boolean): AnyRef = { diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index 12dbebcb28644..e755d2e309398 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -22,6 +22,8 @@ import org.scalatest.FunSuite import org.apache.spark.SparkContext._ import org.apache.spark.util.NonSerializable +import java.io.NotSerializableException + // Common state shared by FailureSuite-launched tasks. We use a global object // for this because any local variables used in the task closures will rightfully // be copied for each task, so there's no other way for them to share state. @@ -102,7 +104,8 @@ class FailureSuite extends FunSuite with LocalSparkContext { results.collect() } assert(thrown.getClass === classOf[SparkException]) - assert(thrown.getMessage.contains("NotSerializableException")) + assert(thrown.getMessage.contains("NotSerializableException") || + thrown.getCause.getClass === classOf[NotSerializableException]) FailureSuiteState.clear() } @@ -116,21 +119,24 @@ class FailureSuite extends FunSuite with LocalSparkContext { sc.parallelize(1 to 10, 2).map(x => a).count() } assert(thrown.getClass === classOf[SparkException]) - assert(thrown.getMessage.contains("NotSerializableException")) + assert(thrown.getMessage.contains("NotSerializableException") || + thrown.getCause.getClass === classOf[NotSerializableException]) // Non-serializable closure in an earlier stage val thrown1 = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() } assert(thrown1.getClass === classOf[SparkException]) - assert(thrown1.getMessage.contains("NotSerializableException")) + assert(thrown1.getMessage.contains("NotSerializableException") || + thrown1.getCause.getClass === classOf[NotSerializableException]) // Non-serializable closure in foreach function val thrown2 = intercept[SparkException] { sc.parallelize(1 to 10, 2).foreach(x => println(a)) } assert(thrown2.getClass === classOf[SparkException]) - assert(thrown2.getMessage.contains("NotSerializableException")) + assert(thrown2.getMessage.contains("NotSerializableException") || + thrown2.getCause.getClass === classOf[NotSerializableException]) FailureSuiteState.clear() } diff --git a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala index 4bd889135631b..8e4a9e2c9f56c 100644 --- a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala @@ -19,9 +19,29 @@ package org.apache.spark import org.scalatest.FunSuite +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ class ImplicitOrderingSuite extends FunSuite with LocalSparkContext { + // Tests that PairRDDFunctions grabs an implicit Ordering in various cases where it should. + test("basic inference of Orderings"){ + sc = new SparkContext("local", "test") + val rdd = sc.parallelize(1 to 10) + + // These RDD methods are in the companion object so that the unserializable ScalaTest Engine + // won't be reachable from the closure object + + // Infer orderings after basic maps to particular types + val basicMapExpectations = ImplicitOrderingSuite.basicMapExpectations(rdd) + basicMapExpectations.map({case (met, explain) => assert(met, explain)}) + + // Infer orderings for other RDD methods + val otherRDDMethodExpectations = ImplicitOrderingSuite.otherRDDMethodExpectations(rdd) + otherRDDMethodExpectations.map({case (met, explain) => assert(met, explain)}) + } +} + +private object ImplicitOrderingSuite { class NonOrderedClass {} class ComparableClass extends Comparable[ComparableClass] { @@ -31,27 +51,36 @@ class ImplicitOrderingSuite extends FunSuite with LocalSparkContext { class OrderedClass extends Ordered[OrderedClass] { override def compare(o: OrderedClass): Int = ??? } - - // Tests that PairRDDFunctions grabs an implicit Ordering in various cases where it should. - test("basic inference of Orderings"){ - sc = new SparkContext("local", "test") - val rdd = sc.parallelize(1 to 10) - - // Infer orderings after basic maps to particular types - assert(rdd.map(x => (x, x)).keyOrdering.isDefined) - assert(rdd.map(x => (1, x)).keyOrdering.isDefined) - assert(rdd.map(x => (x.toString, x)).keyOrdering.isDefined) - assert(rdd.map(x => (null, x)).keyOrdering.isDefined) - assert(rdd.map(x => (new NonOrderedClass, x)).keyOrdering.isEmpty) - assert(rdd.map(x => (new ComparableClass, x)).keyOrdering.isDefined) - assert(rdd.map(x => (new OrderedClass, x)).keyOrdering.isDefined) - - // Infer orderings for other RDD methods - assert(rdd.groupBy(x => x).keyOrdering.isDefined) - assert(rdd.groupBy(x => new NonOrderedClass).keyOrdering.isEmpty) - assert(rdd.groupBy(x => new ComparableClass).keyOrdering.isDefined) - assert(rdd.groupBy(x => new OrderedClass).keyOrdering.isDefined) - assert(rdd.groupBy((x: Int) => x, 5).keyOrdering.isDefined) - assert(rdd.groupBy((x: Int) => x, new HashPartitioner(5)).keyOrdering.isDefined) + + def basicMapExpectations(rdd: RDD[Int]) = { + List((rdd.map(x => (x, x)).keyOrdering.isDefined, + "rdd.map(x => (x, x)).keyOrdering.isDefined"), + (rdd.map(x => (1, x)).keyOrdering.isDefined, + "rdd.map(x => (1, x)).keyOrdering.isDefined"), + (rdd.map(x => (x.toString, x)).keyOrdering.isDefined, + "rdd.map(x => (x.toString, x)).keyOrdering.isDefined"), + (rdd.map(x => (null, x)).keyOrdering.isDefined, + "rdd.map(x => (null, x)).keyOrdering.isDefined"), + (rdd.map(x => (new NonOrderedClass, x)).keyOrdering.isEmpty, + "rdd.map(x => (new NonOrderedClass, x)).keyOrdering.isEmpty"), + (rdd.map(x => (new ComparableClass, x)).keyOrdering.isDefined, + "rdd.map(x => (new ComparableClass, x)).keyOrdering.isDefined"), + (rdd.map(x => (new OrderedClass, x)).keyOrdering.isDefined, + "rdd.map(x => (new OrderedClass, x)).keyOrdering.isDefined")) } -} + + def otherRDDMethodExpectations(rdd: RDD[Int]) = { + List((rdd.groupBy(x => x).keyOrdering.isDefined, + "rdd.groupBy(x => x).keyOrdering.isDefined"), + (rdd.groupBy(x => new NonOrderedClass).keyOrdering.isEmpty, + "rdd.groupBy(x => new NonOrderedClass).keyOrdering.isEmpty"), + (rdd.groupBy(x => new ComparableClass).keyOrdering.isDefined, + "rdd.groupBy(x => new ComparableClass).keyOrdering.isDefined"), + (rdd.groupBy(x => new OrderedClass).keyOrdering.isDefined, + "rdd.groupBy(x => new OrderedClass).keyOrdering.isDefined"), + (rdd.groupBy((x: Int) => x, 5).keyOrdering.isDefined, + "rdd.groupBy((x: Int) => x, 5).keyOrdering.isDefined"), + (rdd.groupBy((x: Int) => x, new HashPartitioner(5)).keyOrdering.isDefined, + "rdd.groupBy((x: Int) => x, new HashPartitioner(5)).keyOrdering.isDefined")) + } +} \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala new file mode 100644 index 0000000000000..5d15a68ac7e4f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.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.serializer; + +import java.io.NotSerializableException + +import org.scalatest.FunSuite + +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkException +import org.apache.spark.SharedSparkContext + +/* A trivial (but unserializable) container for trivial functions */ +class UnserializableClass { + def op[T](x: T) = x.toString + + def pred[T](x: T) = x.toString.length % 2 == 0 +} + +class ProactiveClosureSerializationSuite extends FunSuite with SharedSparkContext { + + def fixture = (sc.parallelize(0 until 1000).map(_.toString), new UnserializableClass) + + test("throws expected serialization exceptions on actions") { + val (data, uc) = fixture + + val ex = intercept[SparkException] { + data.map(uc.op(_)).count + } + + assert(ex.getMessage.contains("Task not serializable")) + } + + // There is probably a cleaner way to eliminate boilerplate here, but we're + // iterating over a map from transformation names to functions that perform that + // transformation on a given RDD, creating one test case for each + + for (transformation <- + Map("map" -> xmap _, "flatMap" -> xflatMap _, "filter" -> xfilter _, + "mapWith" -> xmapWith _, "mapPartitions" -> xmapPartitions _, + "mapPartitionsWithIndex" -> xmapPartitionsWithIndex _, + "mapPartitionsWithContext" -> xmapPartitionsWithContext _, + "filterWith" -> xfilterWith _)) { + val (name, xf) = transformation + + test(s"$name transformations throw proactive serialization exceptions") { + val (data, uc) = fixture + + val ex = intercept[SparkException] { + xf(data, uc) + } + + assert(ex.getMessage.contains("Task not serializable"), + s"RDD.$name doesn't proactively throw NotSerializableException") + } + } + + private def xmap(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.map(y=>uc.op(y)) + private def xmapWith(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.mapWith(x => x.toString)((x,y)=>x + uc.op(y)) + private def xflatMap(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.flatMap(y=>Seq(uc.op(y))) + private def xfilter(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.filter(y=>uc.pred(y)) + private def xfilterWith(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.filterWith(x => x.toString)((x,y)=>uc.pred(y)) + private def xmapPartitions(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.mapPartitions(_.map(y=>uc.op(y))) + private def xmapPartitionsWithIndex(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.mapPartitionsWithIndex((_, it) => it.map(y=>uc.op(y))) + private def xmapPartitionsWithContext(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.mapPartitionsWithContext((_, it) => it.map(y=>uc.op(y))) + +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 4709a62381647..e05db236addca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -532,7 +532,10 @@ abstract class DStream[T: ClassTag] ( * 'this' DStream will be registered as an output stream and therefore materialized. */ def foreachRDD(foreachFunc: (RDD[T], Time) => Unit) { - new ForEachDStream(this, context.sparkContext.clean(foreachFunc)).register() + // because the DStream is reachable from the outer object here, and because + // DStreams can't be serialized with closures, we can't proactively check + // it for serializability and so we pass the optional false to SparkContext.clean + new ForEachDStream(this, context.sparkContext.clean(foreachFunc, false)).register() } /** @@ -540,7 +543,10 @@ abstract class DStream[T: ClassTag] ( * on each RDD of 'this' DStream. */ def transform[U: ClassTag](transformFunc: RDD[T] => RDD[U]): DStream[U] = { - transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r))) + // because the DStream is reachable from the outer object here, and because + // DStreams can't be serialized with closures, we can't proactively check + // it for serializability and so we pass the optional false to SparkContext.clean + transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r), false)) } /** @@ -548,7 +554,10 @@ abstract class DStream[T: ClassTag] ( * on each RDD of 'this' DStream. */ def transform[U: ClassTag](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = { - val cleanedF = context.sparkContext.clean(transformFunc) + // because the DStream is reachable from the outer object here, and because + // DStreams can't be serialized with closures, we can't proactively check + // it for serializability and so we pass the optional false to SparkContext.clean + val cleanedF = context.sparkContext.clean(transformFunc, false) val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => { assert(rdds.length == 1) cleanedF(rdds.head.asInstanceOf[RDD[T]], time) @@ -563,7 +572,10 @@ abstract class DStream[T: ClassTag] ( def transformWith[U: ClassTag, V: ClassTag]( other: DStream[U], transformFunc: (RDD[T], RDD[U]) => RDD[V] ): DStream[V] = { - val cleanedF = ssc.sparkContext.clean(transformFunc) + // because the DStream is reachable from the outer object here, and because + // DStreams can't be serialized with closures, we can't proactively check + // it for serializability and so we pass the optional false to SparkContext.clean + val cleanedF = ssc.sparkContext.clean(transformFunc, false) transformWith(other, (rdd1: RDD[T], rdd2: RDD[U], time: Time) => cleanedF(rdd1, rdd2)) } @@ -574,7 +586,10 @@ abstract class DStream[T: ClassTag] ( def transformWith[U: ClassTag, V: ClassTag]( other: DStream[U], transformFunc: (RDD[T], RDD[U], Time) => RDD[V] ): DStream[V] = { - val cleanedF = ssc.sparkContext.clean(transformFunc) + // because the DStream is reachable from the outer object here, and because + // DStreams can't be serialized with closures, we can't proactively check + // it for serializability and so we pass the optional false to SparkContext.clean + val cleanedF = ssc.sparkContext.clean(transformFunc, false) val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => { assert(rdds.length == 2) val rdd1 = rdds(0).asInstanceOf[RDD[T]] From 6803642253b45c80ef8970606e350adabe4211ea Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Sun, 29 Jun 2014 23:29:05 -0700 Subject: [PATCH 074/124] SPARK-2077 Log serializer that actually ends up being used I could settle with this being a debug also if we provided an example of how to turn it on in `log4j.properties` https://issues.apache.org/jira/browse/SPARK-2077 Author: Andrew Ash Closes #1017 from ash211/SPARK-2077 and squashes the following commits: 580f680 [Andrew Ash] Drop to debug 0266415 [Andrew Ash] SPARK-2077 Log serializer that actually ends up being used --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 8dfa8cc4b5b3f..2b636b085d73a 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -183,6 +183,7 @@ object SparkEnv extends Logging { val serializer = instantiateClass[Serializer]( "spark.serializer", "org.apache.spark.serializer.JavaSerializer") + logDebug(s"Using serializer: ${serializer.getClass}") val closureSerializer = instantiateClass[Serializer]( "spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer") From 358ae1534d01ad9e69364a21441a7ef23c2cb516 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 30 Jun 2014 11:50:22 -0700 Subject: [PATCH 075/124] [SPARK-2322] Exception in resultHandler should NOT crash DAGScheduler and shutdown SparkContext. This should go into 1.0.1. Author: Reynold Xin Closes #1264 from rxin/SPARK-2322 and squashes the following commits: c77c07f [Reynold Xin] Added comment to SparkDriverExecutionException and a test case for accumulator. 5d8d920 [Reynold Xin] [SPARK-2322] Exception in resultHandler could crash DAGScheduler and shutdown SparkContext. --- .../org/apache/spark/SparkException.scala | 8 +++ .../apache/spark/scheduler/DAGScheduler.scala | 20 +++++-- .../spark/scheduler/DAGSchedulerSuite.scala | 56 ++++++++++++++++++- 3 files changed, 78 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkException.scala b/core/src/main/scala/org/apache/spark/SparkException.scala index 4351ed74b67fc..2ebd7a7151a59 100644 --- a/core/src/main/scala/org/apache/spark/SparkException.scala +++ b/core/src/main/scala/org/apache/spark/SparkException.scala @@ -22,3 +22,11 @@ class SparkException(message: String, cause: Throwable) def this(message: String) = this(message, null) } + +/** + * Exception thrown when execution of some user code in the driver process fails, e.g. + * accumulator update fails or failure in takeOrdered (user supplies an Ordering implementation + * that can be misbehaving. + */ +private[spark] class SparkDriverExecutionException(cause: Throwable) + extends SparkException("Execution error", cause) 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 c8559a7a82868..813a9abfafc27 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -581,8 +581,9 @@ class DAGScheduler( } } catch { case e: Exception => - jobResult = JobFailed(e) - job.listener.jobFailed(e) + val exception = new SparkDriverExecutionException(e) + jobResult = JobFailed(exception) + job.listener.jobFailed(exception) case oom: OutOfMemoryError => val exception = new SparkException("Local job aborted due to out of memory error", oom) jobResult = JobFailed(exception) @@ -822,6 +823,7 @@ class DAGScheduler( case Success => logInfo("Completed " + task) if (event.accumUpdates != null) { + // 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 @@ -838,7 +840,16 @@ class DAGScheduler( cleanupStateForJobAndIndependentStages(job, Some(stage)) listenerBus.post(SparkListenerJobEnd(job.jobId, JobSucceeded)) } - job.listener.taskSucceeded(rt.outputId, event.result) + + // taskSucceeded runs some user code that might throw an exception. Make sure + // we are resilient against that. + try { + job.listener.taskSucceeded(rt.outputId, event.result) + } catch { + case e: Exception => + // TODO: Perhaps we want to mark the stage as failed? + job.listener.jobFailed(new SparkDriverExecutionException(e)) + } } case None => logInfo("Ignoring result from " + rt + " because its job has finished") @@ -1161,8 +1172,7 @@ private[scheduler] class DAGSchedulerActorSupervisor(dagScheduler: DAGScheduler) override val supervisorStrategy = OneForOneStrategy() { case x: Exception => - logError("eventProcesserActor failed due to the error %s; shutting down SparkContext" - .format(x.getMessage)) + logError("eventProcesserActor failed; shutting down SparkContext", x) try { dagScheduler.doCancelAllJobs() } catch { 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 8dd2a9b9f7373..9f498d579a095 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.scheduler -import scala.Tuple2 import scala.collection.mutable.{HashSet, HashMap, Map} import scala.language.reflectiveCalls @@ -38,6 +37,8 @@ class BuggyDAGEventProcessActor extends Actor { } } +class DAGSchedulerSuiteDummyException extends Exception + class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with FunSuiteLike with ImplicitSender with BeforeAndAfter with LocalSparkContext { @@ -593,6 +594,59 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assertDataStructuresEmpty } + // TODO: Fix this and un-ignore the test. + ignore("misbehaved accumulator should not crash DAGScheduler and SparkContext") { + val acc = new Accumulator[Int](0, new AccumulatorParam[Int] { + override def addAccumulator(t1: Int, t2: Int): Int = t1 + t2 + override def zero(initialValue: Int): Int = 0 + override def addInPlace(r1: Int, r2: Int): Int = { + throw new DAGSchedulerSuiteDummyException + } + }) + + // Run this on executors + intercept[SparkDriverExecutionException] { + sc.parallelize(1 to 10, 2).foreach { item => acc.add(1) } + } + + // Run this within a local thread + intercept[SparkDriverExecutionException] { + sc.parallelize(1 to 10, 2).map { item => acc.add(1) }.take(1) + } + + // Make sure we can still run local commands as well as cluster commands. + assert(sc.parallelize(1 to 10, 2).count() === 10) + assert(sc.parallelize(1 to 10, 2).first() === 1) + } + + test("misbehaved resultHandler should not crash DAGScheduler and SparkContext") { + val e1 = intercept[SparkDriverExecutionException] { + val rdd = sc.parallelize(1 to 10, 2) + sc.runJob[Int, Int]( + rdd, + (context: TaskContext, iter: Iterator[Int]) => iter.size, + Seq(0), + allowLocal = true, + (part: Int, result: Int) => throw new DAGSchedulerSuiteDummyException) + } + assert(e1.getCause.isInstanceOf[DAGSchedulerSuiteDummyException]) + + val e2 = intercept[SparkDriverExecutionException] { + val rdd = sc.parallelize(1 to 10, 2) + sc.runJob[Int, Int]( + rdd, + (context: TaskContext, iter: Iterator[Int]) => iter.size, + Seq(0, 1), + allowLocal = false, + (part: Int, result: Int) => throw new DAGSchedulerSuiteDummyException) + } + assert(e2.getCause.isInstanceOf[DAGSchedulerSuiteDummyException]) + + // Make sure we can still run local commands as well as cluster commands. + assert(sc.parallelize(1 to 10, 2).count() === 10) + assert(sc.parallelize(1 to 10, 2).first() === 1) + } + test("DAGSchedulerActorSupervisor closes the SparkContext when EventProcessActor crashes") { val actorSystem = ActorSystem("test") val supervisor = actorSystem.actorOf( From 5fccb567b37a94445512c7ec20b830b5e062089f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 30 Jun 2014 15:12:38 -0700 Subject: [PATCH 076/124] [SPARK-2318] When exiting on a signal, print the signal name first. Author: Reynold Xin Closes #1260 from rxin/signalhandler1 and squashes the following commits: 8e73552 [Reynold Xin] Uh add Logging back in ApplicationMaster. 0402ba8 [Reynold Xin] Synchronize SignalLogger.register. dc70705 [Reynold Xin] Added SignalLogger to YARN ApplicationMaster. 79a21b4 [Reynold Xin] Added license header. 0da052c [Reynold Xin] Added the SignalLogger itself. e587d2e [Reynold Xin] [SPARK-2318] When exiting on a signal, print the signal name first. --- .../apache/spark/deploy/master/Master.scala | 5 +- .../apache/spark/deploy/worker/Worker.scala | 5 +- .../CoarseGrainedExecutorBackend.scala | 6 +- .../spark/executor/MesosExecutorBackend.scala | 5 +- .../org/apache/spark/util/SignalLogger.scala | 60 +++++++++++++++++++ .../spark/deploy/yarn/ApplicationMaster.scala | 5 +- .../spark/deploy/yarn/ApplicationMaster.scala | 5 +- 7 files changed, 79 insertions(+), 12 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/SignalLogger.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 33ffcbd216954..11545b8203707 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -41,7 +41,7 @@ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{AkkaUtils, SignalLogger, Utils} private[spark] class Master( host: String, @@ -755,12 +755,13 @@ private[spark] class Master( } } -private[spark] object Master { +private[spark] object Master extends Logging { val systemName = "sparkMaster" private val actorName = "Master" val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r def main(argStrings: Array[String]) { + SignalLogger.register(log) val conf = new SparkConf val args = new MasterArguments(argStrings, conf) val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort, conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index a0ecaf709f8e2..ce425443051b0 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -34,7 +34,7 @@ import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{AkkaUtils, SignalLogger, Utils} /** * @param masterUrls Each url should look like spark://host:port. @@ -365,8 +365,9 @@ private[spark] class Worker( } } -private[spark] object Worker { +private[spark] object Worker extends Logging { def main(argStrings: Array[String]) { + SignalLogger.register(log) val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, args.memory, args.masters, args.workDir) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index b5fd334f40203..8d31bd05fdbec 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.scheduler.TaskDescription import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{AkkaUtils, SignalLogger, Utils} private[spark] class CoarseGrainedExecutorBackend( driverUrl: String, @@ -97,10 +97,12 @@ private[spark] class CoarseGrainedExecutorBackend( } } -private[spark] object CoarseGrainedExecutorBackend { +private[spark] object CoarseGrainedExecutorBackend extends Logging { def run(driverUrl: String, executorId: String, hostname: String, cores: Int, workerUrl: Option[String]) { + SignalLogger.register(log) + SparkHadoopUtil.get.runAsSparkUser { () => // Debug code Utils.checkHost(hostname) diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 74100498bb2bd..2232e6237bf26 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -25,8 +25,8 @@ import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} import org.apache.spark.{Logging, TaskState} import org.apache.spark.TaskState.TaskState -import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.util.{SignalLogger, Utils} private[spark] class MesosExecutorBackend extends MesosExecutor @@ -93,8 +93,9 @@ private[spark] class MesosExecutorBackend /** * Entry point for Mesos executor. */ -private[spark] object MesosExecutorBackend { +private[spark] object MesosExecutorBackend extends Logging { def main(args: Array[String]) { + SignalLogger.register(log) SparkHadoopUtil.get.runAsSparkUser { () => MesosNativeLibrary.load() // Create a new Executor and start it running diff --git a/core/src/main/scala/org/apache/spark/util/SignalLogger.scala b/core/src/main/scala/org/apache/spark/util/SignalLogger.scala new file mode 100644 index 0000000000000..d769b54fa2fae --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/SignalLogger.scala @@ -0,0 +1,60 @@ +/* + * 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 org.apache.commons.lang.SystemUtils +import org.slf4j.Logger +import sun.misc.{Signal, SignalHandler} + +/** + * Used to log signals received. This can be very useful in debugging crashes or kills. + * + * Inspired by Colin Patrick McCabe's similar class from Hadoop. + */ +private[spark] object SignalLogger { + + private var registered = false + + /** Register a signal handler to log signals on UNIX-like systems. */ + def register(log: Logger): Unit = synchronized { + if (SystemUtils.IS_OS_UNIX) { + require(!registered, "Can't re-install the signal handlers") + registered = true + + val signals = Seq("TERM", "HUP", "INT") + for (signal <- signals) { + try { + new SignalLoggerHandler(signal, log) + } catch { + case e: Exception => log.warn("Failed to register signal handler " + signal, e) + } + } + log.info("Registered signal handlers for [" + signals.mkString(", ") + "]") + } + } +} + +private sealed class SignalLoggerHandler(name: String, log: Logger) extends SignalHandler { + + val prevHandler = Signal.handle(new Signal(name), this) + + override def handle(signal: Signal): Unit = { + log.error("RECEIVED SIGNAL " + signal.getNumber() + ": SIG" + signal.getName()) + prevHandler.handle(signal) + } +} diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 1cc9c33cd2d02..438737f7a6b60 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.util.Utils +import org.apache.spark.util.{SignalLogger, Utils} /** * An application master that runs the users driver program and allocates executors. @@ -409,7 +409,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } -object ApplicationMaster { +object ApplicationMaster extends Logging { // Number of times to wait for the allocator loop to complete. // Each loop iteration waits for 100ms, so maximum of 3 seconds. // This is to ensure that we have reasonable number of containers before we start @@ -487,6 +487,7 @@ object ApplicationMaster { } def main(argStrings: Array[String]) { + SignalLogger.register(log) val args = new ApplicationMasterArguments(argStrings) SparkHadoopUtil.get.runAsSparkUser { () => new ApplicationMaster(args).run() diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 6244332f23737..ee1e9c9c23d22 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.webapp.util.WebAppUtils import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.util.Utils +import org.apache.spark.util.{SignalLogger, Utils} /** @@ -363,7 +363,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } -object ApplicationMaster { +object ApplicationMaster extends Logging { // Number of times to wait for the allocator loop to complete. // Each loop iteration waits for 100ms, so maximum of 3 seconds. // This is to ensure that we have reasonable number of containers before we start @@ -455,6 +455,7 @@ object ApplicationMaster { } def main(argStrings: Array[String]) { + SignalLogger.register(log) val args = new ApplicationMasterArguments(argStrings) SparkHadoopUtil.get.runAsSparkUser { () => new ApplicationMaster(args).run() From 04fa1223ee69760f0d23b40e56f4b036aa301879 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 30 Jun 2014 16:03:38 -0700 Subject: [PATCH 077/124] SPARK-2293. Replace RDD.zip usage by map with predict inside. This is the only occurrence of this pattern in the examples that needs to be replaced. It only addresses the example change. Author: Sean Owen Closes #1250 from srowen/SPARK-2293 and squashes the following commits: 6b1b28c [Sean Owen] Compute prediction-and-label RDD directly rather than by zipping, for efficiency --- docs/mllib-naive-bayes.md | 18 ++++++------------ 1 file changed, 6 insertions(+), 12 deletions(-) diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index 4b3a7cab32118..1d1d7dcf6ffcb 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -51,9 +51,8 @@ val training = splits(0) val test = splits(1) val model = NaiveBayes.train(training, lambda = 1.0) -val prediction = model.predict(test.map(_.features)) -val predictionAndLabel = prediction.zip(test.map(_.label)) +val predictionAndLabel = test.map(p => (model.predict(p.features), p.label)) val accuracy = 1.0 * predictionAndLabel.filter(x => x._1 == x._2).count() / test.count() {% endhighlight %} @@ -71,6 +70,7 @@ can be used for evaluation and prediction. import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; import org.apache.spark.mllib.classification.NaiveBayes; import org.apache.spark.mllib.classification.NaiveBayesModel; import org.apache.spark.mllib.regression.LabeledPoint; @@ -81,18 +81,12 @@ JavaRDD test = ... // test set final NaiveBayesModel model = NaiveBayes.train(training.rdd(), 1.0); -JavaRDD prediction = - test.map(new Function() { - @Override public Double call(LabeledPoint p) { - return model.predict(p.features()); - } - }); JavaPairRDD predictionAndLabel = - prediction.zip(test.map(new Function() { - @Override public Double call(LabeledPoint p) { - return p.label(); + test.mapToPair(new PairFunction() { + @Override public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); } - })); + }); double accuracy = 1.0 * predictionAndLabel.filter(new Function, Boolean>() { @Override public Boolean call(Tuple2 pl) { return pl._1() == pl._2(); From 3319a3e3c604f187ff8176597b269af04ca5c1c5 Mon Sep 17 00:00:00 2001 From: Peter MacKinnon Date: Tue, 1 Jul 2014 00:28:38 -0700 Subject: [PATCH 078/124] SPARK-2332 [build] add exclusion for old servlet-api on hadoop-client in core Fix for class of test suite failures in jenkins Author: Peter MacKinnon Closes #1271 from pdmack/master and squashes the following commits: cfe59fd [Peter MacKinnon] exclude servlet-api in hadoop-client for sbt 6f39fec [Peter MacKinnon] add exclusion for old servlet-api on hadoop-client in core --- core/pom.xml | 6 ++++++ project/SparkBuild.scala | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/pom.xml b/core/pom.xml index 8c23842730e37..6abf8480d5da0 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -34,6 +34,12 @@ org.apache.hadoop hadoop-client + + + javax.servlet + servlet-api + + net.java.dev.jets3t diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 55a2aa0fc7141..599714233c18f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -371,7 +371,7 @@ object SparkBuild extends Build { "net.java.dev.jets3t" % "jets3t" % jets3tVersion excludeAll(excludeCommonsLogging), "commons-codec" % "commons-codec" % "1.5", // Prevent jets3t from including the older version of commons-codec "org.apache.derby" % "derby" % "10.4.2.0" % "test", - "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm), + "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm, excludeServletApi), "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeJBossNetty), "com.codahale.metrics" % "metrics-core" % codahaleMetricsVersion, "com.codahale.metrics" % "metrics-jvm" % codahaleMetricsVersion, From 05c3d90e3527114d3abc08c7cc87f6efef96ebdc Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 1 Jul 2014 01:56:51 -0700 Subject: [PATCH 079/124] [SPARK-2185] Emit warning when task size exceeds a threshold. This functionality was added in an earlier commit but shortly after was removed due to a bad git merge (totally my fault). Author: Kay Ousterhout Closes #1149 from kayousterhout/warning_bug and squashes the following commits: 3f1bb00 [Kay Ousterhout] Fixed Json tests 462a664 [Kay Ousterhout] Removed task set name from warning message e89b2f6 [Kay Ousterhout] Fixed Json tests. 7af424c [Kay Ousterhout] Emit warning when task size exceeds a threshold. --- .../apache/spark/scheduler/DAGScheduler.scala | 13 ------ .../apache/spark/scheduler/StageInfo.scala | 2 - .../org/apache/spark/scheduler/TaskInfo.scala | 2 - .../spark/scheduler/TaskSetManager.scala | 15 +++++++ .../org/apache/spark/util/JsonProtocol.scala | 10 +---- .../spark/scheduler/TaskSetManagerSuite.scala | 41 +++++++++++++++++++ .../apache/spark/util/JsonProtocolSuite.scala | 18 ++++---- 7 files changed, 65 insertions(+), 36 deletions(-) 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 813a9abfafc27..81c136d970312 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -623,16 +623,6 @@ class DAGScheduler( } private[scheduler] def handleBeginEvent(task: Task[_], taskInfo: TaskInfo) { - for (stage <- stageIdToStage.get(task.stageId); stageInfo <- stageToInfos.get(stage)) { - if (taskInfo.serializedSize > DAGScheduler.TASK_SIZE_TO_WARN * 1024 && - !stageInfo.emittedTaskSizeWarning) { - stageInfo.emittedTaskSizeWarning = true - logWarning(("Stage %d (%s) contains a task of very large " + - "size (%d KB). The maximum recommended task size is %d KB.").format( - task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, - DAGScheduler.TASK_SIZE_TO_WARN)) - } - } listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) submitWaitingStages() } @@ -1254,7 +1244,4 @@ private[spark] object DAGScheduler { // The time, in millis, to wake up between polls of the completion queue in order to potentially // resubmit failed stages val POLL_TIMEOUT = 10L - - // Warns the user if a stage contains a task with size greater than this value (in KB) - val TASK_SIZE_TO_WARN = 100 } diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 7644e3f351b3c..480891550eb60 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -38,8 +38,6 @@ class StageInfo( /** If the stage failed, the reason why. */ var failureReason: Option[String] = None - var emittedTaskSizeWarning = false - def stageFailed(reason: String) { failureReason = Some(reason) completionTime = Some(System.currentTimeMillis) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 6aecdfe8e6656..29de0453ac19a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -49,8 +49,6 @@ class TaskInfo( var failed = false - var serializedSize: Int = 0 - private[spark] def markGettingResult(time: Long = System.currentTimeMillis) { gettingResultTime = time } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 83ff6b8550b4f..059cc9085a2e7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -166,6 +166,8 @@ private[spark] class TaskSetManager( override def schedulingMode = SchedulingMode.NONE + var emittedTaskSizeWarning = false + /** * Add a task to all the pending-task lists that it should be on. If readding is set, we are * re-adding the task so only include it in each list if it's not already there. @@ -418,6 +420,13 @@ private[spark] class TaskSetManager( // we assume the task can be serialized without exceptions. val serializedTask = Task.serializeWithDependencies( task, sched.sc.addedFiles, sched.sc.addedJars, ser) + if (serializedTask.limit > TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024 && + !emittedTaskSizeWarning) { + emittedTaskSizeWarning = true + logWarning(s"Stage ${task.stageId} contains a task of very large size " + + s"(${serializedTask.limit / 1024} KB). The maximum recommended task size is " + + s"${TaskSetManager.TASK_SIZE_TO_WARN_KB} KB.") + } val timeTaken = clock.getTime() - startTime addRunningTask(taskId) logInfo("Serialized task %s:%d as %d bytes in %d ms".format( @@ -764,3 +773,9 @@ private[spark] class TaskSetManager( localityWaits = myLocalityLevels.map(getLocalityWait) } } + +private[spark] object TaskSetManager { + // The user will be warned if any stages contain a task that has a serialized size greater than + // this. + val TASK_SIZE_TO_WARN_KB = 100 +} diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 26c9c9d6034ec..47eb44b530379 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -190,8 +190,7 @@ private[spark] object JsonProtocol { ("Details" -> stageInfo.details) ~ ("Submission Time" -> submissionTime) ~ ("Completion Time" -> completionTime) ~ - ("Failure Reason" -> failureReason) ~ - ("Emitted Task Size Warning" -> stageInfo.emittedTaskSizeWarning) + ("Failure Reason" -> failureReason) } def taskInfoToJson(taskInfo: TaskInfo): JValue = { @@ -205,8 +204,7 @@ private[spark] object JsonProtocol { ("Speculative" -> taskInfo.speculative) ~ ("Getting Result Time" -> taskInfo.gettingResultTime) ~ ("Finish Time" -> taskInfo.finishTime) ~ - ("Failed" -> taskInfo.failed) ~ - ("Serialized Size" -> taskInfo.serializedSize) + ("Failed" -> taskInfo.failed) } def taskMetricsToJson(taskMetrics: TaskMetrics): JValue = { @@ -487,13 +485,11 @@ private[spark] object JsonProtocol { val submissionTime = Utils.jsonOption(json \ "Submission Time").map(_.extract[Long]) val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]) val failureReason = Utils.jsonOption(json \ "Failure Reason").map(_.extract[String]) - val emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfos, details) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime stageInfo.failureReason = failureReason - stageInfo.emittedTaskSizeWarning = emittedTaskSizeWarning stageInfo } @@ -509,14 +505,12 @@ private[spark] object JsonProtocol { val gettingResultTime = (json \ "Getting Result Time").extract[Long] val finishTime = (json \ "Finish Time").extract[Long] val failed = (json \ "Failed").extract[Boolean] - val serializedSize = (json \ "Serialized Size").extract[Int] val taskInfo = new TaskInfo(taskId, index, attempt, launchTime, executorId, host, taskLocality, speculative) taskInfo.gettingResultTime = gettingResultTime taskInfo.finishTime = finishTime taskInfo.failed = failed - taskInfo.serializedSize = serializedSize taskInfo } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 59a618956a356..9ff2a487005c4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import java.util.Random + import scala.collection.mutable.ArrayBuffer import scala.collection.mutable @@ -83,6 +85,18 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex } } +/** + * A Task implementation that results in a large serialized task. + */ +class LargeTask(stageId: Int) extends Task[Array[Byte]](stageId, 0) { + val randomBuffer = new Array[Byte](TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024) + val random = new Random(0) + random.nextBytes(randomBuffer) + + override def runTask(context: TaskContext): Array[Byte] = randomBuffer + override def preferredLocations: Seq[TaskLocation] = Seq[TaskLocation]() +} + class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { import TaskLocality.{ANY, PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL} @@ -434,6 +448,33 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) } + test("do not emit warning when serialized task is small") { + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + val taskSet = FakeTask.createTaskSet(1) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) + + assert(!manager.emittedTaskSizeWarning) + + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + + assert(!manager.emittedTaskSizeWarning) + } + + test("emit warning when serialized task is large") { + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + + val taskSet = new TaskSet(Array(new LargeTask(0)), 0, 0, 0, null) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) + + assert(!manager.emittedTaskSizeWarning) + + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) + + assert(manager.emittedTaskSizeWarning) + } + def createTaskResult(id: Int): DirectTaskResult[Int] = { val valueSer = SparkEnv.get.serializer.newInstance() new DirectTaskResult[Int](valueSer.serialize(id), mutable.Map.empty, new TaskMetrics) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 316e14100e409..058d31453081a 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -257,7 +257,6 @@ class JsonProtocolSuite extends FunSuite { assert(info1.numTasks === info2.numTasks) assert(info1.submissionTime === info2.submissionTime) assert(info1.completionTime === info2.completionTime) - assert(info1.emittedTaskSizeWarning === info2.emittedTaskSizeWarning) assert(info1.rddInfos.size === info2.rddInfos.size) (0 until info1.rddInfos.size).foreach { i => assertEquals(info1.rddInfos(i), info2.rddInfos(i)) @@ -294,7 +293,6 @@ class JsonProtocolSuite extends FunSuite { assert(info1.gettingResultTime === info2.gettingResultTime) assert(info1.finishTime === info2.finishTime) assert(info1.failed === info2.failed) - assert(info1.serializedSize === info2.serializedSize) } private def assertEquals(metrics1: TaskMetrics, metrics2: TaskMetrics) { @@ -540,9 +538,8 @@ class JsonProtocolSuite extends FunSuite { private val stageSubmittedJsonString = """ {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":100,"Stage Name": - "greetings","Number of Tasks":200,"RDD Info":[],"Details":"details", - "Emitted Task Size Warning":false},"Properties":{"France":"Paris","Germany":"Berlin", - "Russia":"Moscow","Ukraine":"Kiev"}} + "greetings","Number of Tasks":200,"RDD Info":[],"Details":"details"},"Properties": + {"France":"Paris","Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} """ private val stageCompletedJsonString = @@ -551,8 +548,7 @@ class JsonProtocolSuite extends FunSuite { "greetings","Number of Tasks":201,"RDD Info":[{"RDD ID":101,"Name":"mayor","Storage Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":true, "Replication":1},"Number of Partitions":201,"Number of Cached Partitions":301, - "Memory Size":401,"Tachyon Size":0,"Disk Size":501}],"Details":"details", - "Emitted Task Size Warning":false}} + "Memory Size":401,"Tachyon Size":0,"Disk Size":501}],"Details":"details"}} """ private val taskStartJsonString = @@ -560,7 +556,7 @@ class JsonProtocolSuite extends FunSuite { |{"Event":"SparkListenerTaskStart","Stage ID":111,"Task Info":{"Task ID":222, |"Index":333,"Attempt":1,"Launch Time":444,"Executor ID":"executor","Host":"your kind sir", |"Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0, - |"Failed":false,"Serialized Size":0}} + |"Failed":false}} """.stripMargin private val taskGettingResultJsonString = @@ -568,7 +564,7 @@ class JsonProtocolSuite extends FunSuite { |{"Event":"SparkListenerTaskGettingResult","Task Info": | {"Task ID":1000,"Index":2000,"Attempt":5,"Launch Time":3000,"Executor ID":"executor", | "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":true,"Getting Result Time":0, - | "Finish Time":0,"Failed":false,"Serialized Size":0 + | "Finish Time":0,"Failed":false | } |} """.stripMargin @@ -580,7 +576,7 @@ class JsonProtocolSuite extends FunSuite { |"Task Info":{ | "Task ID":123,"Index":234,"Attempt":67,"Launch Time":345,"Executor ID":"executor", | "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":false, - | "Getting Result Time":0,"Finish Time":0,"Failed":false,"Serialized Size":0 + | "Getting Result Time":0,"Finish Time":0,"Failed":false |}, |"Task Metrics":{ | "Host Name":"localhost","Executor Deserialize Time":300,"Executor Run Time":400, @@ -620,7 +616,7 @@ class JsonProtocolSuite extends FunSuite { |"Task Info":{ | "Task ID":123,"Index":234,"Attempt":67,"Launch Time":345,"Executor ID":"executor", | "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":false, - | "Getting Result Time":0,"Finish Time":0,"Failed":false,"Serialized Size":0 + | "Getting Result Time":0,"Finish Time":0,"Failed":false |}, |"Task Metrics":{ | "Host Name":"localhost","Executor Deserialize Time":300,"Executor Run Time":400, From 6596392da0fc0fee89e22adfca239a3477dfcbab Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 1 Jul 2014 20:37:10 -0700 Subject: [PATCH 080/124] update the comments in SqlParser SqlParser has been case-insensitive after https://github.com/apache/spark/commit/dab5439a083b5f771d5d5b462d0d517fa8e9aaf2 was merged Author: CodingCat Closes #1275 from CodingCat/master and squashes the following commits: 17931cd [CodingCat] update the comments in SqlParser --- .../src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 0cc4592047b19..61762fa2a7c30 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -35,7 +35,6 @@ import org.apache.spark.sql.catalyst.types._ * * Limitations: * - Only supports a very limited subset of SQL. - * - Keywords must be capital. * * This is currently included mostly for illustrative purposes. Users wanting more complete support * for a SQL like language should checkout the HiveQL support in the sql/hive sub-project. From 5c6ec94da1bacd8e65a43acb92b6721493484e7b Mon Sep 17 00:00:00 2001 From: Ximo Guanter Gonzalbez Date: Wed, 2 Jul 2014 10:03:44 -0700 Subject: [PATCH 081/124] SPARK-2186: Spark SQL DSL support for simple aggregations such as SUM and AVG **Description** This patch enables using the `.select()` function in SchemaRDD with functions such as `Sum`, `Count` and other. **Testing** Unit tests added. Author: Ximo Guanter Gonzalbez Closes #1211 from edrevo/add-expression-support-in-select and squashes the following commits: fe4a1e1 [Ximo Guanter Gonzalbez] Extend SQL DSL to functions e1d344a [Ximo Guanter Gonzalbez] SPARK-2186: Spark SQL DSL support for simple aggregations such as SUM and AVG --- .../spark/sql/catalyst/dsl/package.scala | 11 +++++++ .../org/apache/spark/sql/SchemaRDD.scala | 9 ++++-- .../org/apache/spark/sql/DslQuerySuite.scala | 32 +++++++++++++++---- 3 files changed, 44 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 26ad4837b0b01..1b503b957d146 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -108,6 +108,17 @@ package object dsl { implicit def symbolToUnresolvedAttribute(s: Symbol) = analysis.UnresolvedAttribute(s.name) + def sum(e: Expression) = Sum(e) + def sumDistinct(e: Expression) = SumDistinct(e) + def count(e: Expression) = Count(e) + def countDistinct(e: Expression*) = CountDistinct(e) + def avg(e: Expression) = Average(e) + def first(e: Expression) = First(e) + def min(e: Expression) = Min(e) + def max(e: Expression) = Max(e) + def upper(e: Expression) = Upper(e) + def lower(e: Expression) = Lower(e) + implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s = sym.name } // TODO more implicit class for literal? implicit class DslString(val s: String) extends ImplicitOperators { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 7c0efb4566610..8f9f54f610e9d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -133,8 +133,13 @@ class SchemaRDD( * * @group Query */ - def select(exprs: NamedExpression*): SchemaRDD = - new SchemaRDD(sqlContext, Project(exprs, logicalPlan)) + def select(exprs: Expression*): SchemaRDD = { + val aliases = exprs.zipWithIndex.map { + case (ne: NamedExpression, _) => ne + case (e, i) => Alias(e, s"c$i")() + } + new SchemaRDD(sqlContext, Project(aliases, logicalPlan)) + } /** * Filters the output, only returning those rows where `condition` evaluates to true. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index e4a64a7a482b8..04ac008682f5f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -60,6 +60,26 @@ class DslQuerySuite extends QueryTest { Seq(Seq("1"))) } + test("select with functions") { + checkAnswer( + testData.select(sum('value), avg('value), count(1)), + Seq(Seq(5050.0, 50.5, 100))) + + checkAnswer( + testData2.select('a + 'b, 'a < 'b), + Seq( + Seq(2, false), + Seq(3, true), + Seq(3, false), + Seq(4, false), + Seq(4, false), + Seq(5, false))) + + checkAnswer( + testData2.select(sumDistinct('a)), + Seq(Seq(6))) + } + test("sorting") { checkAnswer( testData2.orderBy('a.asc, 'b.asc), @@ -110,17 +130,17 @@ class DslQuerySuite extends QueryTest { test("average") { checkAnswer( - testData2.groupBy()(Average('a)), + testData2.groupBy()(avg('a)), 2.0) } test("null average") { checkAnswer( - testData3.groupBy()(Average('b)), + testData3.groupBy()(avg('b)), 2.0) checkAnswer( - testData3.groupBy()(Average('b), CountDistinct('b :: Nil)), + testData3.groupBy()(avg('b), countDistinct('b)), (2.0, 1) :: Nil) } @@ -130,17 +150,17 @@ class DslQuerySuite extends QueryTest { test("null count") { checkAnswer( - testData3.groupBy('a)('a, Count('b)), + testData3.groupBy('a)('a, count('b)), Seq((1,0), (2, 1)) ) checkAnswer( - testData3.groupBy('a)('a, Count('a + 'b)), + testData3.groupBy('a)('a, count('a + 'b)), Seq((1,0), (2, 1)) ) checkAnswer( - testData3.groupBy()(Count('a), Count('b), Count(1), CountDistinct('a :: Nil), CountDistinct('b :: Nil)), + testData3.groupBy()(count('a), count('b), count(1), countDistinct('a), countDistinct('b)), (2, 1, 2, 2, 1) :: Nil ) } From 1e2c26c83dd2e807cf0031ceca8b338a1a57cac6 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Wed, 2 Jul 2014 10:07:01 -0700 Subject: [PATCH 082/124] [SPARK-2328] [SQL] Add execution of `SHOW TABLES` before `TestHive.reset()`. `PruningSuite` is executed first of Hive tests unfortunately, `TestHive.reset()` breaks the test environment. To prevent this, we must run a query before calling reset the first time. Author: Takuya UESHIN Closes #1268 from ueshin/issues/SPARK-2328 and squashes the following commits: 043ceac [Takuya UESHIN] Add execution of `SHOW TABLES` before `TestHive.reset()`. --- .../org/apache/spark/sql/hive/execution/PruningSuite.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 4d7c84f443879..34d8a061ccc83 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -26,6 +26,9 @@ import scala.collection.JavaConversions._ * A set of test cases that validate partition and column pruning. */ class PruningSuite extends HiveComparisonTest { + // MINOR HACK: You must run a query before calling reset the first time. + TestHive.hql("SHOW TABLES") + // Column/partition pruning is not implemented for `InMemoryColumnarTableScan` yet, need to reset // the environment to ensure all referenced tables in this suites are not cached in-memory. // Refer to https://issues.apache.org/jira/browse/SPARK-2283 for details. From bc7041a42dfa84312492ea8cae6fdeaeac4f6d1c Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Wed, 2 Jul 2014 10:10:36 -0700 Subject: [PATCH 083/124] [SPARK-2287] [SQL] Make ScalaReflection be able to handle Generic case classes. Author: Takuya UESHIN Closes #1226 from ueshin/issues/SPARK-2287 and squashes the following commits: 32ef7c3 [Takuya UESHIN] Add execution of `SHOW TABLES` before `TestHive.reset()`. 541dc8d [Takuya UESHIN] Merge branch 'master' into issues/SPARK-2287 fac5fae [Takuya UESHIN] Remove unnecessary method receiver. d306e60 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-2287 7de5706 [Takuya UESHIN] Make ScalaReflection be able to handle Generic case classes. --- .../spark/sql/catalyst/ScalaReflection.scala | 7 +++++-- .../sql/catalyst/ScalaReflectionSuite.scala | 20 +++++++++++++++++++ 2 files changed, 25 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index ada48eaf5dc0f..5a55be1e51558 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -47,10 +47,13 @@ object ScalaReflection { val TypeRef(_, _, Seq(optType)) = t Schema(schemaFor(optType).dataType, nullable = true) case t if t <:< typeOf[Product] => - val params = t.member("": TermName).asMethod.paramss + val formalTypeArgs = t.typeSymbol.asClass.typeParams + val TypeRef(_, _, actualTypeArgs) = t + val params = t.member(nme.CONSTRUCTOR).asMethod.paramss Schema(StructType( params.head.map { p => - val Schema(dataType, nullable) = schemaFor(p.typeSignature) + val Schema(dataType, nullable) = + schemaFor(p.typeSignature.substituteTypes(formalTypeArgs, actualTypeArgs)) StructField(p.name.toString, dataType, nullable) }), nullable = true) // Need to decide if we actually need a special type here. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index 489d7e9c2437f..c0438dbe52a47 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -60,6 +60,9 @@ case class ComplexData( mapField: Map[Int, String], structField: PrimitiveData) +case class GenericData[A]( + genericField: A) + class ScalaReflectionSuite extends FunSuite { import ScalaReflection._ @@ -128,4 +131,21 @@ class ScalaReflectionSuite extends FunSuite { nullable = true))), nullable = true)) } + + test("generic data") { + val schema = schemaFor[GenericData[Int]] + assert(schema === Schema( + StructType(Seq( + StructField("genericField", IntegerType, nullable = false))), + nullable = true)) + } + + test("tuple data") { + val schema = schemaFor[(Int, String)] + assert(schema === Schema( + StructType(Seq( + StructField("_1", IntegerType, nullable = false), + StructField("_2", StringType, nullable = true))), + nullable = true)) + } } From 3bbeca648985b32bdf1eedef779cb2817eb6dfa4 Mon Sep 17 00:00:00 2001 From: yantangzhai Date: Thu, 3 Jul 2014 10:14:35 -0700 Subject: [PATCH 084/124] [SPARK-2324] SparkContext should not exit directly when spark.local.dir is a list of multiple paths and one of them has error The spark.local.dir is configured as a list of multiple paths as follows /data1/sparkenv/local,/data2/sparkenv/local. If the disk data2 of the driver node has error, the application will exit since DiskBlockManager exits directly at createLocalDirs. If the disk data2 of the worker node has error, the executor will exit either. DiskBlockManager should not exit directly at createLocalDirs if one of spark.local.dir has error. Since spark.local.dir has multiple paths, a problem should not affect the overall situation. I think DiskBlockManager could ignore the bad directory at createLocalDirs. Author: yantangzhai Closes #1274 from YanTangZhai/SPARK-2324 and squashes the following commits: 609bf48 [yantangzhai] [SPARK-2324] SparkContext should not exit directly when spark.local.dir is a list of multiple paths and one of them has error df08673 [yantangzhai] [SPARK-2324] SparkContext should not exit directly when spark.local.dir is a list of multiple paths and one of them has error --- .../apache/spark/storage/DiskBlockManager.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 2ec46d416f37d..673fc19c060a4 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -44,6 +44,10 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD * directory, create multiple subdirectories that we will hash files into, in order to avoid * having really large inodes at the top level. */ private val localDirs: Array[File] = createLocalDirs() + if (localDirs.isEmpty) { + logError("Failed to create any local dir.") + System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) + } private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) private var shuffleSender : ShuffleSender = null @@ -116,7 +120,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD private def createLocalDirs(): Array[File] = { logDebug(s"Creating local directories at root dirs '$rootDirs'") val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") - rootDirs.split(",").map { rootDir => + rootDirs.split(",").flatMap { rootDir => var foundLocalDir = false var localDir: File = null var localDirId: String = null @@ -136,11 +140,13 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD } } if (!foundLocalDir) { - logError(s"Failed $MAX_DIR_CREATION_ATTEMPTS attempts to create local dir in $rootDir") - System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) + logError(s"Failed $MAX_DIR_CREATION_ATTEMPTS attempts to create local dir in $rootDir." + + " Ignoring this directory.") + None + } else { + logInfo(s"Created local directory at $localDir") + Some(localDir) } - logInfo(s"Created local directory at $localDir") - localDir } } From c480537739f9329ebfd580f09c69778e6c976366 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 3 Jul 2014 10:26:50 -0700 Subject: [PATCH 085/124] [SPARK] Fix NPE for ExternalAppendOnlyMap It did not handle null keys very gracefully before. Author: Andrew Or Closes #1288 from andrewor14/fix-external and squashes the following commits: 312b8d8 [Andrew Or] Abstract key hash code ed5adf9 [Andrew Or] Fix NPE for ExternalAppendOnlyMap --- .../collection/ExternalAppendOnlyMap.scala | 30 ++++++++++++++----- .../ExternalAppendOnlyMapSuite.scala | 27 +++++++++++++++-- 2 files changed, 46 insertions(+), 11 deletions(-) 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 288badd3160f8..292d0962f4fdb 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 @@ -252,7 +252,7 @@ class ExternalAppendOnlyMap[K, V, C]( if (it.hasNext) { var kc = it.next() kcPairs += kc - val minHash = kc._1.hashCode() + val minHash = getKeyHashCode(kc) while (it.hasNext && it.head._1.hashCode() == minHash) { kc = it.next() kcPairs += kc @@ -294,8 +294,9 @@ class ExternalAppendOnlyMap[K, V, C]( // Select a key from the StreamBuffer that holds the lowest key hash val minBuffer = mergeHeap.dequeue() val (minPairs, minHash) = (minBuffer.pairs, minBuffer.minKeyHash) - var (minKey, minCombiner) = minPairs.remove(0) - assert(minKey.hashCode() == minHash) + val minPair = minPairs.remove(0) + var (minKey, minCombiner) = minPair + assert(getKeyHashCode(minPair) == minHash) // For all other streams that may have this key (i.e. have the same minimum key hash), // merge in the corresponding value (if any) from that stream @@ -327,15 +328,16 @@ class ExternalAppendOnlyMap[K, V, C]( * StreamBuffers are ordered by the minimum key hash found across all of their own pairs. */ private class StreamBuffer( - val iterator: BufferedIterator[(K, C)], val pairs: ArrayBuffer[(K, C)]) + val iterator: BufferedIterator[(K, C)], + val pairs: ArrayBuffer[(K, C)]) extends Comparable[StreamBuffer] { def isEmpty = pairs.length == 0 // Invalid if there are no more pairs in this stream - def minKeyHash = { + def minKeyHash: Int = { assert(pairs.length > 0) - pairs.head._1.hashCode() + getKeyHashCode(pairs.head) } override def compareTo(other: StreamBuffer): Int = { @@ -422,10 +424,22 @@ class ExternalAppendOnlyMap[K, V, C]( } private[spark] object ExternalAppendOnlyMap { + + /** + * Return the key hash code of the given (key, combiner) pair. + * If the key is null, return a special hash code. + */ + private def getKeyHashCode[K, C](kc: (K, C)): Int = { + if (kc._1 == null) 0 else kc._1.hashCode() + } + + /** + * A comparator for (key, combiner) pairs based on their key hash codes. + */ private class KCComparator[K, C] extends Comparator[(K, C)] { def compare(kc1: (K, C), kc2: (K, C)): Int = { - val hash1 = kc1._1.hashCode() - val hash2 = kc2._1.hashCode() + val hash1 = getKeyHashCode(kc1) + val hash2 = getKeyHashCode(kc2) if (hash1 < hash2) -1 else if (hash1 == hash2) 0 else 1 } } 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 deb780953579d..428822949c085 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 @@ -334,8 +334,8 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) - val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, - mergeValue, mergeCombiners) + val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]]( + createCombiner, mergeValue, mergeCombiners) (1 to 100000).foreach { i => map.insert(i, i) } map.insert(Int.MaxValue, Int.MaxValue) @@ -346,11 +346,32 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { it.next() } } + + test("spilling with null keys and values") { + val conf = new SparkConf(true) + conf.set("spark.shuffle.memoryFraction", "0.001") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) + + val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]]( + createCombiner, mergeValue, mergeCombiners) + + (1 to 100000).foreach { i => map.insert(i, i) } + map.insert(null.asInstanceOf[Int], 1) + map.insert(1, null.asInstanceOf[Int]) + map.insert(null.asInstanceOf[Int], null.asInstanceOf[Int]) + + val it = map.iterator + while (it.hasNext) { + // Should not throw NullPointerException + it.next() + } + } + } /** * A dummy class that always returns the same hash code, to easily test hash collisions */ -case class FixedHashObject(val v: Int, val h: Int) extends Serializable { +case class FixedHashObject(v: Int, h: Int) extends Serializable { override def hashCode(): Int = h } From 2b36344f588d4e7357ce9921dc656e2389ba1dea Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 3 Jul 2014 11:54:51 -0700 Subject: [PATCH 086/124] SPARK-1675. Make clear whether computePrincipalComponents requires centered data Just closing out this small JIRA, resolving with a comment change. Author: Sean Owen Closes #1171 from srowen/SPARK-1675 and squashes the following commits: 45ee9b7 [Sean Owen] Add simple note that data need not be centered for computePrincipalComponents --- .../org/apache/spark/mllib/linalg/distributed/RowMatrix.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 1a0073c9d487e..695e03b736baf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -347,6 +347,8 @@ class RowMatrix( * The principal components are stored a local matrix of size n-by-k. * Each column corresponds for one principal component, * and the columns are in descending order of component variance. + * The row data do not need to be "centered" first; it is not necessary for + * the mean of each column to be 0. * * @param k number of top principal components. * @return a matrix of size n-by-k, whose columns are principal components From a9b52e5623f7fc77fca96b095f9eeaef76e35d54 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Thu, 3 Jul 2014 13:22:13 -0700 Subject: [PATCH 087/124] [SPARK-2342] Evaluation helper's output type doesn't conform to input ty... The function cast doesn't conform to the intention of "Those expressions are supposed to be in the same data type, and also the return type." comment Author: Yijie Shen Closes #1283 from yijieshen/master and squashes the following commits: c7aaa4b [Yijie Shen] [SPARK-2342] Evaluation helper's output type doesn't conform to input type --- .../org/apache/spark/sql/catalyst/expressions/Expression.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 0411ce3aefda1..ba62dabe3dd6a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -111,7 +111,7 @@ abstract class Expression extends TreeNode[Expression] { } else { e1.dataType match { case n: NumericType => - f.asInstanceOf[(Numeric[n.JvmType], n.JvmType, n.JvmType) => Int]( + f.asInstanceOf[(Numeric[n.JvmType], n.JvmType, n.JvmType) => n.JvmType]( n.numeric, evalE1.asInstanceOf[n.JvmType], evalE2.asInstanceOf[n.JvmType]) case other => sys.error(s"Type $other does not support numeric operations") } From 731f683b1bd8abbb83030b6bae14876658bbf098 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 3 Jul 2014 15:06:58 -0700 Subject: [PATCH 088/124] [SPARK-2109] Setting SPARK_MEM for bin/pyspark does not work. Trivial fix. Author: Prashant Sharma Closes #1050 from ScrapCodes/SPARK-2109/pyspark-script-bug and squashes the following commits: 77072b9 [Prashant Sharma] Changed echos to redirect to STDERR. 13f48a0 [Prashant Sharma] [SPARK-2109] Setting SPARK_MEM for bin/pyspark does not work. --- bin/compute-classpath.sh | 8 ++++---- bin/pyspark | 6 +++--- bin/run-example | 10 +++++----- bin/spark-class | 13 ++++++------- 4 files changed, 18 insertions(+), 19 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 2cf4e381c1c88..e81e8c060cb98 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -81,10 +81,10 @@ ASSEMBLY_JAR=$(ls "$assembly_folder"/spark-assembly*hadoop*.jar 2>/dev/null) # Verify that versions of java used to build the jars and run Spark are compatible jar_error_check=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" nonexistent/class/path 2>&1) if [[ "$jar_error_check" =~ "invalid CEN header" ]]; then - echo "Loading Spark jar with '$JAR_CMD' failed. " - echo "This is likely because Spark was compiled with Java 7 and run " - echo "with Java 6. (see SPARK-1703). Please use Java 7 to run Spark " - echo "or build Spark with Java 6." + echo "Loading Spark jar with '$JAR_CMD' failed. " 1>&2 + echo "This is likely because Spark was compiled with Java 7 and run " 1>&2 + echo "with Java 6. (see SPARK-1703). Please use Java 7 to run Spark " 1>&2 + echo "or build Spark with Java 6." 1>&2 exit 1 fi diff --git a/bin/pyspark b/bin/pyspark index 0b5ed40e2157d..69b056fe28f2c 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -26,7 +26,7 @@ export SPARK_HOME="$FWDIR" SCALA_VERSION=2.10 if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - echo "Usage: ./bin/pyspark [options]" + echo "Usage: ./bin/pyspark [options]" 1>&2 $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 exit 0 fi @@ -36,8 +36,8 @@ if [ ! -f "$FWDIR/RELEASE" ]; then # Exit if the user hasn't compiled Spark ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null if [[ $? != 0 ]]; then - echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2 - echo "You need to build Spark before running this program" >&2 + echo "Failed to find Spark assembly in $FWDIR/assembly/target" 1>&2 + echo "You need to build Spark before running this program" 1>&2 exit 1 fi fi diff --git a/bin/run-example b/bin/run-example index e7a5fe3914fbd..942706d733122 100755 --- a/bin/run-example +++ b/bin/run-example @@ -27,9 +27,9 @@ if [ -n "$1" ]; then EXAMPLE_CLASS="$1" shift else - echo "Usage: ./bin/run-example [example-args]" - echo " - set MASTER=XX to use a specific master" - echo " - can use abbreviated example class name (e.g. SparkPi, mllib.LinearRegression)" + echo "Usage: ./bin/run-example [example-args]" 1>&2 + echo " - set MASTER=XX to use a specific master" 1>&2 + echo " - can use abbreviated example class name (e.g. SparkPi, mllib.LinearRegression)" 1>&2 exit 1 fi @@ -40,8 +40,8 @@ elif [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.ja fi if [[ -z $SPARK_EXAMPLES_JAR ]]; then - echo "Failed to find Spark examples assembly in $FWDIR/lib or $FWDIR/examples/target" >&2 - echo "You need to build Spark before running this program" >&2 + echo "Failed to find Spark examples assembly in $FWDIR/lib or $FWDIR/examples/target" 1>&2 + echo "You need to build Spark before running this program" 1>&2 exit 1 fi diff --git a/bin/spark-class b/bin/spark-class index 60d9657c0ffcd..04fa52c6756b1 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -33,13 +33,13 @@ export SPARK_HOME="$FWDIR" . $FWDIR/bin/load-spark-env.sh if [ -z "$1" ]; then - echo "Usage: spark-class []" >&2 + echo "Usage: spark-class []" 1>&2 exit 1 fi if [ -n "$SPARK_MEM" ]; then - echo "Warning: SPARK_MEM is deprecated, please use a more specific config option" - echo "(e.g., spark.executor.memory or SPARK_DRIVER_MEMORY)." + echo -e "Warning: SPARK_MEM is deprecated, please use a more specific config option" 1>&2 + echo -e "(e.g., spark.executor.memory or SPARK_DRIVER_MEMORY)." 1>&2 fi # Use SPARK_MEM or 512m as the default memory, to be overridden by specific options @@ -147,10 +147,9 @@ fi export CLASSPATH if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then - echo -n "Spark Command: " - echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" - echo "========================================" - echo + echo -n "Spark Command: " 1>&2 + echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" 1>&2 + echo -e "========================================\n" 1>&2 fi exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" From d4c30cd9918e18dde2a52909e36eaef6eb5996ab Mon Sep 17 00:00:00 2001 From: Zongheng Yang Date: Thu, 3 Jul 2014 17:37:53 -0700 Subject: [PATCH 089/124] [HOTFIX] Synchronize on SQLContext.settings in tests. Let's see if this fixes the ongoing series of test failures in a master build machine (https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-SBT-pre-YARN/SPARK_HADOOP_VERSION=1.0.4,label=centos/81/). pwendell marmbrus Author: Zongheng Yang Closes #1277 from concretevitamin/test-fix and squashes the following commits: 28c88bd [Zongheng Yang] Synchronize on SQLContext.settings in tests. --- .../scala/org/apache/spark/sql/SQLConf.scala | 2 +- .../org/apache/spark/sql/JoinSuite.scala | 40 +++++------ .../org/apache/spark/sql/SQLConfSuite.scala | 64 +++++++++-------- .../org/apache/spark/sql/SQLQuerySuite.scala | 68 ++++++++++--------- 4 files changed, 91 insertions(+), 83 deletions(-) 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 2fe7f94663996..3b5abab969861 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 @@ -50,7 +50,7 @@ trait SQLConf { /** ********************** SQLConf functionality methods ************ */ @transient - private val settings = java.util.Collections.synchronizedMap( + protected[sql] val settings = java.util.Collections.synchronizedMap( new java.util.HashMap[String, String]()) def set(props: Properties): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 3d7d5eedbe8ed..054b14f8f7ffa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -39,25 +39,27 @@ class JoinSuite extends QueryTest { test("plans broadcast hash join, given hints") { def mkTest(buildSide: BuildSide, leftTable: String, rightTable: String) = { - TestSQLContext.set("spark.sql.join.broadcastTables", - s"${if (buildSide == BuildRight) rightTable else leftTable}") - val rdd = sql(s"""SELECT * FROM $leftTable JOIN $rightTable ON key = a""") - // Using `sparkPlan` because for relevant patterns in HashJoin to be - // matched, other strategies need to be applied. - val physical = rdd.queryExecution.sparkPlan - val bhj = physical.collect { case j: BroadcastHashJoin if j.buildSide == buildSide => j } - - assert(bhj.size === 1, "planner does not pick up hint to generate broadcast hash join") - checkAnswer( - rdd, - Seq( - (1, "1", 1, 1), - (1, "1", 1, 2), - (2, "2", 2, 1), - (2, "2", 2, 2), - (3, "3", 3, 1), - (3, "3", 3, 2) - )) + TestSQLContext.settings.synchronized { + TestSQLContext.set("spark.sql.join.broadcastTables", + s"${if (buildSide == BuildRight) rightTable else leftTable}") + val rdd = sql( s"""SELECT * FROM $leftTable JOIN $rightTable ON key = a""") + // Using `sparkPlan` because for relevant patterns in HashJoin to be + // matched, other strategies need to be applied. + val physical = rdd.queryExecution.sparkPlan + val bhj = physical.collect { case j: BroadcastHashJoin if j.buildSide == buildSide => j} + + assert(bhj.size === 1, "planner does not pick up hint to generate broadcast hash join") + checkAnswer( + rdd, + Seq( + (1, "1", 1, 1), + (1, "1", 1, 2), + (2, "2", 2, 1), + (2, "2", 2, 2), + (3, "3", 3, 1), + (3, "3", 3, 2) + )) + } } mkTest(BuildRight, "testData", "testData2") 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 08293f7f0ca30..93792f698cfaf 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 @@ -28,46 +28,50 @@ class SQLConfSuite extends QueryTest { val testVal = "test.val.0" test("programmatic ways of basic setting and getting") { - clear() - assert(getOption(testKey).isEmpty) - assert(getAll.toSet === Set()) + TestSQLContext.settings.synchronized { + clear() + assert(getOption(testKey).isEmpty) + assert(getAll.toSet === Set()) - set(testKey, testVal) - assert(get(testKey) == testVal) - assert(get(testKey, testVal + "_") == testVal) - assert(getOption(testKey) == Some(testVal)) - assert(contains(testKey)) + set(testKey, testVal) + assert(get(testKey) == testVal) + assert(get(testKey, testVal + "_") == testVal) + assert(getOption(testKey) == Some(testVal)) + assert(contains(testKey)) - // Tests SQLConf as accessed from a SQLContext is mutable after - // the latter is initialized, unlike SparkConf inside a SparkContext. - assert(TestSQLContext.get(testKey) == testVal) - assert(TestSQLContext.get(testKey, testVal + "_") == testVal) - assert(TestSQLContext.getOption(testKey) == Some(testVal)) - assert(TestSQLContext.contains(testKey)) + // Tests SQLConf as accessed from a SQLContext is mutable after + // the latter is initialized, unlike SparkConf inside a SparkContext. + assert(TestSQLContext.get(testKey) == testVal) + assert(TestSQLContext.get(testKey, testVal + "_") == testVal) + assert(TestSQLContext.getOption(testKey) == Some(testVal)) + assert(TestSQLContext.contains(testKey)) - clear() + clear() + } } test("parse SQL set commands") { - clear() - sql(s"set $testKey=$testVal") - assert(get(testKey, testVal + "_") == testVal) - assert(TestSQLContext.get(testKey, testVal + "_") == testVal) + TestSQLContext.settings.synchronized { + clear() + sql(s"set $testKey=$testVal") + assert(get(testKey, testVal + "_") == testVal) + assert(TestSQLContext.get(testKey, testVal + "_") == testVal) - 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") + 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" - sql(s"set $key=$vs") - assert(get(key, "0") == vs) + val key = "spark.sql.key" + val vs = "val0,val_1,val2.3,my_table" + sql(s"set $key=$vs") + assert(get(key, "0") == vs) - sql(s"set $key=") - assert(get(key, "0") == "") + sql(s"set $key=") + assert(get(key, "0") == "") - clear() + clear() + } } } 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 bf7fafe952303..2c1cb1867010c 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 @@ -372,38 +372,40 @@ class SQLQuerySuite extends QueryTest { } test("SET commands semantics using sql()") { - clear() - val testKey = "test.key.0" - val testVal = "test.val.0" - val nonexistentKey = "nonexistent" - - // "set" itself returns all config variables currently specified in SQLConf. - assert(sql("SET").collect().size == 0) - - // "set key=val" - sql(s"SET $testKey=$testVal") - checkAnswer( - sql("SET"), - Seq(Seq(testKey, testVal)) - ) - - sql(s"SET ${testKey + testKey}=${testVal + testVal}") - checkAnswer( - sql("set"), - Seq( - Seq(testKey, testVal), - Seq(testKey + testKey, testVal + testVal)) - ) - - // "set key" - checkAnswer( - sql(s"SET $testKey"), - Seq(Seq(testKey, testVal)) - ) - checkAnswer( - sql(s"SET $nonexistentKey"), - Seq(Seq(nonexistentKey, "")) - ) - clear() + TestSQLContext.settings.synchronized { + clear() + val testKey = "test.key.0" + val testVal = "test.val.0" + val nonexistentKey = "nonexistent" + + // "set" itself returns all config variables currently specified in SQLConf. + assert(sql("SET").collect().size == 0) + + // "set key=val" + sql(s"SET $testKey=$testVal") + checkAnswer( + sql("SET"), + Seq(Seq(testKey, testVal)) + ) + + sql(s"SET ${testKey + testKey}=${testVal + testVal}") + checkAnswer( + sql("set"), + Seq( + Seq(testKey, testVal), + Seq(testKey + testKey, testVal + testVal)) + ) + + // "set key" + checkAnswer( + sql(s"SET $testKey"), + Seq(Seq(testKey, testVal)) + ) + checkAnswer( + sql(s"SET $nonexistentKey"), + Seq(Seq(nonexistentKey, "")) + ) + clear() + } } } From fdc4c112e7c2ac585d108d03209a642aa8bab7c8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Cl=C3=A9ment=20MATHIEU?= Date: Thu, 3 Jul 2014 18:31:18 -0700 Subject: [PATCH 090/124] Streaming programming guide typos MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Fix a bad Java code sample and a broken link in the streaming programming guide. Author: Clément MATHIEU Closes #1286 from cykl/streaming-programming-guide-typos and squashes the following commits: b0908cb [Clément MATHIEU] Fix broken URL 9d3c535 [Clément MATHIEU] Spark streaming requires at least two working threads (scala version was OK) --- docs/streaming-programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index ce8e58d64a7ed..90a0eef60c200 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -148,7 +148,7 @@ import org.apache.spark.streaming.*; import org.apache.spark.streaming.api.java.*; import scala.Tuple2; // Create a StreamingContext with a local master -JavaStreamingContext jssc = new JavaStreamingContext("local", "JavaNetworkWordCount", new Duration(1000)) +JavaStreamingContext jssc = new JavaStreamingContext("local[2]", "JavaNetworkWordCount", new Duration(1000)) {% endhighlight %} Using this context, we then create a new DStream @@ -216,7 +216,7 @@ jssc.awaitTermination(); // Wait for the computation to terminate {% endhighlight %} The complete code can be found in the Spark Streaming example -[JavaNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/index.html?org/apache/spark/examples/streaming/JavaNetworkWordCount.java). +[JavaNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java).
    From 5fa0a05763ab1d527efe20e3b10539ac5ffc36de Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Thu, 3 Jul 2014 19:24:22 -0700 Subject: [PATCH 091/124] [SPARK-1097] Workaround Hadoop conf ConcurrentModification issue Workaround Hadoop conf ConcurrentModification issue Author: Raymond Liu Closes #1273 from colorant/hadoopRDD and squashes the following commits: 994e98b [Raymond Liu] Address comments e2cda3d [Raymond Liu] Workaround Hadoop conf ConcurrentModification issue --- core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 98dcbf4e2dbfa..041028514399b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -141,8 +141,8 @@ class HadoopRDD[K, V]( // local process. The local cache is accessed through HadoopRDD.putCachedMetadata(). // The caching helps minimize GC, since a JobConf can contain ~10KB of temporary objects. // synchronize to prevent ConcurrentModificationException (Spark-1097, Hadoop-10456) - broadcastedConf.synchronized { - val newJobConf = new JobConf(broadcastedConf.value.value) + conf.synchronized { + val newJobConf = new JobConf(conf) initLocalJobConfFuncOpt.map(f => f(newJobConf)) HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf) newJobConf From 586feb5c9528042420f678f78bacb6c254a5eaf8 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 3 Jul 2014 22:31:41 -0700 Subject: [PATCH 092/124] [SPARK-2350] Don't NPE while launching drivers Prior to this change, we could throw a NPE if we launch a driver while another one is waiting, because removing from an iterator while iterating over it is not safe. Author: Aaron Davidson Closes #1289 from aarondav/master-fail and squashes the following commits: 1cf1cf4 [Aaron Davidson] SPARK-2350: Don't NPE while launching drivers --- core/src/main/scala/org/apache/spark/deploy/master/Master.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 11545b8203707..a304102a49086 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -481,7 +481,7 @@ private[spark] class Master( // First schedule drivers, they take strict precedence over applications val shuffledWorkers = Random.shuffle(workers) // Randomization helps balance drivers for (worker <- shuffledWorkers if worker.state == WorkerState.ALIVE) { - for (driver <- waitingDrivers) { + for (driver <- List(waitingDrivers: _*)) { // iterate over a copy of waitingDrivers if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { launchDriver(worker, driver) waitingDrivers -= driver From 3894a49be9b532cc026d908a0f49bca850504498 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 3 Jul 2014 22:48:23 -0700 Subject: [PATCH 093/124] [SPARK-2307][Reprise] Correctly report RDD blocks on SparkUI **Problem.** The existing code in `ExecutorPage.scala` requires a linear scan through all the blocks to filter out the uncached ones. Every refresh could be expensive if there are many blocks and many executors. **Solution.** The proper semantics should be the following: `StorageStatusListener` should contain only block statuses that are cached. This means as soon as a block is unpersisted by any mean, its status should be removed. This is reflected in the changes made in `StorageStatusListener.scala`. Further, the `StorageTab` must stop relying on the `StorageStatusListener` changing a dropped block's status to `StorageLevel.NONE` (which no longer happens). This is reflected in the changes made in `StorageTab.scala` and `StorageUtils.scala`. ---------- If you have been following this chain of PRs like pwendell, you will quickly notice that this reverts the changes in #1249, which reverts the changes in #1080. In other words, we are adding back the changes from #1080, and fixing SPARK-2307 on top of those changes. Please ask questions if you are confused. Author: Andrew Or Closes #1255 from andrewor14/storage-ui-fix-reprise and squashes the following commits: 45416fa [Andrew Or] Merge branch 'master' of github.com:apache/spark into storage-ui-fix-reprise a82ea25 [Andrew Or] Add tests for StorageStatusListener 8773b01 [Andrew Or] Update comment / minor changes 3afde3f [Andrew Or] Correctly report the number of blocks on SparkUI --- .../spark/storage/StorageStatusListener.scala | 17 +- .../apache/spark/storage/StorageUtils.scala | 15 +- .../apache/spark/ui/exec/ExecutorsPage.scala | 4 +- .../apache/spark/ui/exec/ExecutorsTab.scala | 4 +- .../apache/spark/ui/storage/StorageTab.scala | 15 +- .../storage/StorageStatusListenerSuite.scala | 152 ++++++++++++++++++ 6 files changed, 184 insertions(+), 23 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index a6e6627d54e01..41c960c867e2e 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -28,26 +28,31 @@ import org.apache.spark.scheduler._ */ @DeveloperApi class StorageStatusListener extends SparkListener { - private val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() + // This maintains only blocks that are cached (i.e. storage level is not StorageLevel.NONE) + private[storage] val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() def storageStatusList = executorIdToStorageStatus.values.toSeq /** Update storage status list to reflect updated block statuses */ - def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) { - val filteredStatus = storageStatusList.find(_.blockManagerId.executorId == execId) + private def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) { + val filteredStatus = executorIdToStorageStatus.get(execId) filteredStatus.foreach { storageStatus => updatedBlocks.foreach { case (blockId, updatedStatus) => - storageStatus.blocks(blockId) = updatedStatus + if (updatedStatus.storageLevel == StorageLevel.NONE) { + storageStatus.blocks.remove(blockId) + } else { + storageStatus.blocks(blockId) = updatedStatus + } } } } /** Update storage status list to reflect the removal of an RDD from the cache */ - def updateStorageStatus(unpersistedRDDId: Int) { + private def updateStorageStatus(unpersistedRDDId: Int) { storageStatusList.foreach { storageStatus => val unpersistedBlocksIds = storageStatus.rddBlocks.keys.filter(_.rddId == unpersistedRDDId) unpersistedBlocksIds.foreach { blockId => - storageStatus.blocks(blockId) = BlockStatus(StorageLevel.NONE, 0L, 0L, 0L) + storageStatus.blocks.remove(blockId) } } } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index f3bde1df45c79..177281f663367 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -75,17 +75,26 @@ private[spark] object StorageUtils { /** Returns storage information of all RDDs in the given list. */ def rddInfoFromStorageStatus( storageStatuses: Seq[StorageStatus], - rddInfos: Seq[RDDInfo]): Array[RDDInfo] = { + rddInfos: Seq[RDDInfo], + updatedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty): Array[RDDInfo] = { + + // Mapping from a block ID -> its status + val blockMap = mutable.Map(storageStatuses.flatMap(_.rddBlocks): _*) + + // Record updated blocks, if any + updatedBlocks + .collect { case (id: RDDBlockId, status) => (id, status) } + .foreach { case (id, status) => blockMap(id) = status } // Mapping from RDD ID -> an array of associated BlockStatuses - val blockStatusMap = storageStatuses.flatMap(_.rddBlocks).toMap + val rddBlockMap = blockMap .groupBy { case (k, _) => k.rddId } .mapValues(_.values.toArray) // Mapping from RDD ID -> the associated RDDInfo (with potentially outdated storage information) val rddInfoMap = rddInfos.map { info => (info.id, info) }.toMap - val rddStorageInfos = blockStatusMap.flatMap { case (rddId, blocks) => + val rddStorageInfos = rddBlockMap.flatMap { case (rddId, blocks) => // Add up memory, disk and Tachyon sizes val persistedBlocks = blocks.filter { status => status.memSize + status.diskSize + status.tachyonSize > 0 } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 9625337ae21a5..95b4a4e91d333 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -110,9 +110,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { val status = listener.storageStatusList(statusId) val execId = status.blockManagerId.executorId val hostPort = status.blockManagerId.hostPort - val rddBlocks = status.blocks.count { case (_, blockStatus) => - blockStatus.storageLevel != StorageLevel.NONE - } + val rddBlocks = status.blocks.size val memUsed = status.memUsed val maxMem = status.maxMem val diskUsed = status.diskUsed diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 58eeb86bf9a3a..5c2d1d1fe75d3 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -39,9 +39,7 @@ private[ui] class ExecutorsTab(parent: SparkUI) extends WebUITab(parent, "execut * A SparkListener that prepares information to be displayed on the ExecutorsTab */ @DeveloperApi -class ExecutorsListener(storageStatusListener: StorageStatusListener) - extends SparkListener { - +class ExecutorsListener(storageStatusListener: StorageStatusListener) extends SparkListener { val executorToTasksActive = HashMap[String, Int]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index c4bb7aab50393..0cc0cf3117173 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -22,7 +22,7 @@ import scala.collection.mutable import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ui._ import org.apache.spark.scheduler._ -import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} +import org.apache.spark.storage._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[ui] class StorageTab(parent: SparkUI) extends WebUITab(parent, "storage") { @@ -40,9 +40,7 @@ private[ui] class StorageTab(parent: SparkUI) extends WebUITab(parent, "storage" * A SparkListener that prepares information to be displayed on the BlockManagerUI. */ @DeveloperApi -class StorageListener(storageStatusListener: StorageStatusListener) - extends SparkListener { - +class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener { private val _rddInfoMap = mutable.Map[Int, RDDInfo]() def storageStatusList = storageStatusListener.storageStatusList @@ -51,9 +49,10 @@ class StorageListener(storageStatusListener: StorageStatusListener) def rddInfoList = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq /** Update each RDD's info to reflect any updates to the RDD's storage status */ - private def updateRDDInfo() { + private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty) { val rddInfos = _rddInfoMap.values.toSeq - val updatedRddInfos = StorageUtils.rddInfoFromStorageStatus(storageStatusList, rddInfos) + val updatedRddInfos = + StorageUtils.rddInfoFromStorageStatus(storageStatusList, rddInfos, updatedBlocks) updatedRddInfos.foreach { info => _rddInfoMap(info.id) = info } } @@ -64,7 +63,7 @@ class StorageListener(storageStatusListener: StorageStatusListener) override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { val metrics = taskEnd.taskMetrics if (metrics != null && metrics.updatedBlocks.isDefined) { - updateRDDInfo() + updateRDDInfo(metrics.updatedBlocks.get) } } @@ -79,6 +78,6 @@ class StorageListener(storageStatusListener: StorageStatusListener) } override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) = synchronized { - updateRDDInfo() + _rddInfoMap.remove(unpersistRDD.rddId) } } diff --git a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala new file mode 100644 index 0000000000000..2179c6dd3302e --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import org.scalatest.FunSuite +import org.apache.spark.Success +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler._ + +/** + * Test the behavior of StorageStatusListener in response to all relevant events. + */ +class StorageStatusListenerSuite extends FunSuite { + private val bm1 = BlockManagerId("big", "dog", 1, 1) + private val bm2 = BlockManagerId("fat", "duck", 2, 2) + private val taskInfo1 = new TaskInfo(0, 0, 0, 0, "big", "dog", TaskLocality.ANY, false) + private val taskInfo2 = new TaskInfo(0, 0, 0, 0, "fat", "duck", TaskLocality.ANY, false) + + test("block manager added/removed") { + val listener = new StorageStatusListener + + // Block manager add + assert(listener.executorIdToStorageStatus.size === 0) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm1, 1000L)) + assert(listener.executorIdToStorageStatus.size === 1) + assert(listener.executorIdToStorageStatus.get("big").isDefined) + assert(listener.executorIdToStorageStatus("big").blockManagerId === bm1) + assert(listener.executorIdToStorageStatus("big").maxMem === 1000L) + assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm2, 2000L)) + assert(listener.executorIdToStorageStatus.size === 2) + assert(listener.executorIdToStorageStatus.get("fat").isDefined) + assert(listener.executorIdToStorageStatus("fat").blockManagerId === bm2) + assert(listener.executorIdToStorageStatus("fat").maxMem === 2000L) + assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + + // Block manager remove + listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(bm1)) + assert(listener.executorIdToStorageStatus.size === 1) + assert(!listener.executorIdToStorageStatus.get("big").isDefined) + assert(listener.executorIdToStorageStatus.get("fat").isDefined) + listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(bm2)) + assert(listener.executorIdToStorageStatus.size === 0) + assert(!listener.executorIdToStorageStatus.get("big").isDefined) + assert(!listener.executorIdToStorageStatus.get("fat").isDefined) + } + + test("task end without updated blocks") { + val listener = new StorageStatusListener + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm1, 1000L)) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm2, 2000L)) + val taskMetrics = new TaskMetrics + + // Task end with no updated blocks + assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics)) + assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics)) + assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + } + + test("task end with updated blocks") { + val listener = new StorageStatusListener + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm1, 1000L)) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm2, 2000L)) + val taskMetrics1 = new TaskMetrics + val taskMetrics2 = new TaskMetrics + val block1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.DISK_ONLY, 0L, 100L, 0L)) + val block2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.DISK_ONLY, 0L, 200L, 0L)) + val block3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.DISK_ONLY, 0L, 300L, 0L)) + taskMetrics1.updatedBlocks = Some(Seq(block1, block2)) + taskMetrics2.updatedBlocks = Some(Seq(block3)) + + // Task end with new blocks + assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) + assert(listener.executorIdToStorageStatus("big").blocks.size === 2) + assert(listener.executorIdToStorageStatus("fat").blocks.size === 0) + assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics2)) + assert(listener.executorIdToStorageStatus("big").blocks.size === 2) + assert(listener.executorIdToStorageStatus("fat").blocks.size === 1) + assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("fat").blocks.contains(RDDBlockId(4, 0))) + + // Task end with dropped blocks + val droppedBlock1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) + val droppedBlock2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) + val droppedBlock3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) + taskMetrics1.updatedBlocks = Some(Seq(droppedBlock1, droppedBlock3)) + taskMetrics2.updatedBlocks = Some(Seq(droppedBlock2, droppedBlock3)) + listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) + assert(listener.executorIdToStorageStatus("big").blocks.size === 1) + assert(listener.executorIdToStorageStatus("fat").blocks.size === 1) + assert(!listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("fat").blocks.contains(RDDBlockId(4, 0))) + listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics2)) + assert(listener.executorIdToStorageStatus("big").blocks.size === 1) + assert(listener.executorIdToStorageStatus("fat").blocks.size === 0) + assert(!listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + } + + test("unpersist RDD") { + val listener = new StorageStatusListener + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm1, 1000L)) + val taskMetrics1 = new TaskMetrics + val taskMetrics2 = new TaskMetrics + val block1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.DISK_ONLY, 0L, 100L, 0L)) + val block2 = (RDDBlockId(1, 2), BlockStatus(StorageLevel.DISK_ONLY, 0L, 200L, 0L)) + val block3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.DISK_ONLY, 0L, 300L, 0L)) + taskMetrics1.updatedBlocks = Some(Seq(block1, block2)) + taskMetrics2.updatedBlocks = Some(Seq(block3)) + listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) + listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics2)) + assert(listener.executorIdToStorageStatus("big").blocks.size === 3) + + // Unpersist RDD + listener.onUnpersistRDD(SparkListenerUnpersistRDD(9090)) + assert(listener.executorIdToStorageStatus("big").blocks.size === 3) + listener.onUnpersistRDD(SparkListenerUnpersistRDD(4)) + assert(listener.executorIdToStorageStatus("big").blocks.size === 2) + assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) + listener.onUnpersistRDD(SparkListenerUnpersistRDD(1)) + assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) + } +} From 97a0bfe1c0261384f09d53f9350de52fb6446d59 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 3 Jul 2014 23:02:36 -0700 Subject: [PATCH 094/124] SPARK-2282: Reuse PySpark Accumulator sockets to avoid crashing Spark JIRA: https://issues.apache.org/jira/browse/SPARK-2282 This issue is caused by a buildup of sockets in the TIME_WAIT stage of TCP, which is a stage that lasts for some period of time after the communication closes. This solution simply allows us to reuse sockets that are in TIME_WAIT, to avoid issues with the buildup of the rapid creation of these sockets. Author: Aaron Davidson Closes #1220 from aarondav/SPARK-2282 and squashes the following commits: 2e5cab3 [Aaron Davidson] SPARK-2282: Reuse PySpark Accumulator sockets to avoid crashing Spark --- core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala | 2 ++ 1 file changed, 2 insertions(+) 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 f6570d335757a..462e09466bfa6 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 @@ -599,6 +599,8 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: } else { // This happens on the master, where we pass the updates to Python through a socket val socket = new Socket(serverHost, serverPort) + // SPARK-2282: Immediately reuse closed sockets because we create one per task. + socket.setReuseAddress(true) val in = socket.getInputStream val out = new DataOutputStream(new BufferedOutputStream(socket.getOutputStream, bufferSize)) out.writeInt(val2.size) From 544880457de556d1ad52e8cb7e1eca19da95f517 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 3 Jul 2014 23:41:54 -0700 Subject: [PATCH 095/124] [SPARK-2059][SQL] Don't throw TreeNodeException in `execution.ExplainCommand` This is a fix for the problem revealed by PR #1265. Currently `HiveComparisonSuite` ignores output of `ExplainCommand` since Catalyst query plan is quite different from Hive query plan. But exceptions throw from `CheckResolution` still breaks test cases. This PR catches any `TreeNodeException` and reports it as part of the query explanation. After merging this PR, PR #1265 can also be merged safely. For a normal query: ``` scala> hql("explain select key from src").foreach(println) ... [Physical execution plan:] [HiveTableScan [key#9], (MetastoreRelation default, src, None), None] ``` For a wrong query with unresolved attribute(s): ``` scala> hql("explain select kay from src").foreach(println) ... [Error occurred during query planning: ] [Unresolved attributes: 'kay, tree:] [Project ['kay]] [ LowerCaseSchema ] [ MetastoreRelation default, src, None] ``` Author: Cheng Lian Closes #1294 from liancheng/safe-explain and squashes the following commits: 4318911 [Cheng Lian] Don't throw TreeNodeException in `execution.ExplainCommand` --- .../scala/org/apache/spark/sql/execution/commands.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) 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 acb1b0f4dc229..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 @@ -19,9 +19,10 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{SQLContext, Row} -import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute} +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, SQLContext} trait Command { /** @@ -86,8 +87,10 @@ case class ExplainCommand( extends LeafNode with Command { // Run through the optimizer to generate the physical plan. - override protected[sql] lazy val sideEffectResult: Seq[String] = { + override protected[sql] lazy val sideEffectResult: Seq[String] = try { "Physical execution plan:" +: context.executePlan(logicalPlan).executedPlan.toString.split("\n") + } catch { case cause: TreeNodeException[_] => + "Error occurred during query planning: " +: cause.getMessage.split("\n") } def execute(): RDD[Row] = { From d43415075b3468fe8aa56de5d2907d409bb96347 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 4 Jul 2014 00:05:27 -0700 Subject: [PATCH 096/124] [SPARK-1199][REPL] Remove VALId and use the original import style for defined classes. This is an alternate solution to #1176. Author: Prashant Sharma Closes #1179 from ScrapCodes/SPARK-1199/repl-fix-second-approach and squashes the following commits: 820b34b [Prashant Sharma] Here we generate two kinds of import wrappers based on whether it is a class or not. --- .../org/apache/spark/repl/SparkIMain.scala | 7 ++++-- .../org/apache/spark/repl/SparkImports.scala | 23 +++++++++++-------- .../org/apache/spark/repl/ReplSuite.scala | 12 ++++++++++ 3 files changed, 31 insertions(+), 11 deletions(-) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala index 7c83fa9d4d888..3842c291d0b7b 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -744,7 +744,7 @@ import org.apache.spark.util.Utils * * Read! Eval! Print! Some of that not yet centralized here. */ - class ReadEvalPrint(lineId: Int) { + class ReadEvalPrint(val lineId: Int) { def this() = this(freshLineId()) private var lastRun: Run = _ @@ -1241,7 +1241,10 @@ import org.apache.spark.util.Utils // old style beSilentDuring(parse(code)) foreach { ts => ts foreach { t => - withoutUnwrapping(logDebug(asCompactString(t))) + if (isShow || isShowRaw) + withoutUnwrapping(echo(asCompactString(t))) + else + withoutUnwrapping(logDebug(asCompactString(t))) } } } diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala index 419796b68b113..bce5c74b9d0da 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala @@ -182,15 +182,26 @@ trait SparkImports { // ambiguity errors will not be generated. Also, quote // the name of the variable, so that we don't need to // handle quoting keywords separately. + case x: ClassHandler => + // I am trying to guess if the import is a defined class + // This is an ugly hack, I am not 100% sure of the consequences. + // Here we, let everything but "defined classes" use the import with val. + // The reason for this is, otherwise the remote executor tries to pull the + // classes involved and may fail. + for (imv <- x.definedNames) { + val objName = req.lineRep.readPath + code.append("import " + objName + ".INSTANCE" + req.accessPath + ".`" + imv + "`\n") + } + case x => for (imv <- x.definedNames) { if (currentImps contains imv) addWrapper() val objName = req.lineRep.readPath - val valName = "$VAL" + newValId(); + val valName = "$VAL" + req.lineRep.lineId if(!code.toString.endsWith(".`" + imv + "`;\n")) { // Which means already imported - code.append("val " + valName + " = " + objName + ".INSTANCE;\n") - code.append("import " + valName + req.accessPath + ".`" + imv + "`;\n") + code.append("val " + valName + " = " + objName + ".INSTANCE;\n") + code.append("import " + valName + req.accessPath + ".`" + imv + "`;\n") } // code.append("val " + valName + " = " + objName + ".INSTANCE;\n") // code.append("import " + valName + req.accessPath + ".`" + imv + "`;\n") @@ -211,10 +222,4 @@ trait SparkImports { private def membersAtPickler(sym: Symbol): List[Symbol] = beforePickler(sym.info.nonPrivateMembers.toList) - private var curValId = 0 - - private def newValId(): Int = { - curValId += 1 - curValId - } } diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index f4ba8d9cc079b..f2aa42dbcb4fc 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -235,6 +235,18 @@ class ReplSuite extends FunSuite { assertContains("res4: Array[Int] = Array(0, 0, 0, 0, 0)", output) } + test("SPARK-1199-simple-reproduce") { + val output = runInterpreter("local-cluster[1,1,512]", + """ + |case class Sum(exp: String, exp2: String) + |val a = Sum("A", "B") + |def b(a: Sum): String = a match { case Sum(_, _) => "Found Sum" } + |b(a) + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + } + if (System.getenv("MESOS_NATIVE_LIBRARY") != null) { test("running on Mesos") { val output = runInterpreter("localquiet", From 0bbe61223eda3f33bbf8992d2a8f0d47813f4873 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?baishuo=28=E7=99=BD=E7=A1=95=29?= Date: Fri, 4 Jul 2014 00:25:31 -0700 Subject: [PATCH 097/124] Update SQLConf.scala MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit use concurrent.ConcurrentHashMap instead of util.Collections.synchronizedMap Author: baishuo(白硕) Closes #1272 from baishuo/master and squashes the following commits: 51ec55d [baishuo(白硕)] Update SQLConf.scala 63da043 [baishuo(白硕)] Update SQLConf.scala 36b6dbd [baishuo(白硕)] Update SQLConf.scala 864faa0 [baishuo(白硕)] Update SQLConf.scala 593096b [baishuo(白硕)] Update SQLConf.scala 7304d9b [baishuo(白硕)] Update SQLConf.scala 843581c [baishuo(白硕)] Update SQLConf.scala 1d3e4a2 [baishuo(白硕)] Update SQLConf.scala 0740f28 [baishuo(白硕)] Update SQLConf.scala --- .../src/main/scala/org/apache/spark/sql/SQLConf.scala | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) 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 3b5abab969861..95ed0f28507fc 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 @@ -64,20 +64,17 @@ trait SQLConf { } def get(key: String): String = { - if (!settings.containsKey(key)) { - throw new NoSuchElementException(key) - } - settings.get(key) + Option(settings.get(key)).getOrElse(throw new NoSuchElementException(key)) } def get(key: String, defaultValue: String): String = { - if (!settings.containsKey(key)) defaultValue else settings.get(key) + Option(settings.get(key)).getOrElse(defaultValue) } def getAll: Array[(String, String)] = settings.asScala.toArray def getOption(key: String): Option[String] = { - if (!settings.containsKey(key)) None else Some(settings.get(key)) + Option(settings.get(key)) } def contains(key: String): Boolean = settings.containsKey(key) From b3e768e154bd7175db44c3ffc3d8f783f15ab776 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 4 Jul 2014 00:53:41 -0700 Subject: [PATCH 098/124] [SPARK-2059][SQL] Add analysis checks This replaces #1263 with a test case. Author: Reynold Xin Author: Michael Armbrust Closes #1265 from rxin/sql-analysis-error and squashes the following commits: a639e01 [Reynold Xin] Added a test case for unresolved attribute analysis. 7371e1b [Reynold Xin] Merge pull request #1263 from marmbrus/analysisChecks 448c088 [Michael Armbrust] Add analysis checks --- .../spark/sql/catalyst/analysis/Analyzer.scala | 16 ++++++++++++++++ .../sql/catalyst/analysis/AnalysisSuite.scala | 8 ++++++++ 2 files changed, 24 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 4ebc0e70d946b..c7188469bfb86 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ @@ -54,10 +55,25 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool ResolveFunctions :: GlobalAggregates :: typeCoercionRules :_*), + Batch("Check Analysis", Once, + CheckResolution), Batch("AnalysisOperators", fixedPoint, EliminateAnalysisOperators) ) + /** + * Makes sure all attributes have been resolved. + */ + object CheckResolution extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = { + plan.transform { + case p if p.expressions.exists(!_.resolved) => + throw new TreeNodeException(p, + s"Unresolved attributes: ${p.expressions.filterNot(_.resolved).mkString(",")}") + } + } + } + /** * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 4c313585c6386..f14df8137683b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical._ /* Implicit conversions */ @@ -34,4 +35,11 @@ class AnalysisSuite extends FunSuite { analyze(Project(Seq(UnresolvedAttribute("a")), testRelation)) === Project(testRelation.output, testRelation)) } + + test("throw errors for unresolved attributes during analysis") { + val e = intercept[TreeNodeException[_]] { + analyze(Project(Seq(UnresolvedAttribute("abcd")), testRelation)) + } + assert(e.getMessage().toLowerCase.contains("unresolved")) + } } From 5dadda86456e1d3918e320b83aec7e2f1352d95d Mon Sep 17 00:00:00 2001 From: Yanjie Gao Date: Fri, 4 Jul 2014 02:43:57 -0700 Subject: [PATCH 099/124] [SPARK-2234][SQL]Spark SQL basicOperators add Except operator Hi all, I want to submit a Except operator in basicOperators.scala In SQL case.SQL support two table do except operator. select * from table1 except select * from table2 This operator support the substract function .Return an table with the elements from `this` that are not in `other`.This operator should limit the input SparkPlan Seq only has two member.The check will later support JIRA:https://issues.apache.org/jira/browse/SPARK-2234 Author: Yanjie Gao Author: YanjieGao <396154235@qq.com> Author: root Author: gaoyanjie Closes #1151 from YanjieGao/patch-6 and squashes the following commits: f19f899 [YanjieGao] add a new blank line in basicoperators.scala 2ff7d73 [YanjieGao] resolve the identation in SqlParser and SparkStrategies fdb5227 [YanjieGao] Merge remote branch 'upstream/master' into patch-6 9940d19 [YanjieGao] make comment less than 100c 09c7413 [YanjieGao] pr 1151 SqlParser add cache ,basic Operator rename Except and modify comment b4b5867 [root] Merge remote branch 'upstream/master' into patch-6 b4c3869 [Yanjie Gao] change SparkStrategies Sparkcontext to SqlContext 7e0ec29 [Yanjie Gao] delete multi test 7e7c83f [Yanjie Gao] delete conflict except b01beb8 [YanjieGao] resolve conflict sparkstrategies and basicOperators 4dc8166 [YanjieGao] resolve conflict fa68a98 [Yanjie Gao] Update joins.scala 8e6bb00 [Yanjie Gao] delete conflict except dd9ba5e [Yanjie Gao] Update joins.scala a0d4e73 [Yanjie Gao] delete skew join 60f5ddd [Yanjie Gao] update less than 100c 0e72233 [Yanjie Gao] update SQLQuerySuite on master branch 7f916b5 [Yanjie Gao] update execution/basicOperators on master branch a28dece [Yanjie Gao] Update logical/basicOperators on master branch a639935 [Yanjie Gao] Update SparkStrategies.scala 3bf7def [Yanjie Gao] update SqlParser on master branch 26f833f [Yanjie Gao] update SparkStrategies.scala on master branch 8dd063f [Yanjie Gao] Update logical/basicOperators on master branch 9847dcf [Yanjie Gao] update SqlParser on masterbranch d6a4604 [Yanjie Gao] Update joins.scala 424c507 [Yanjie Gao] Update joins.scala 7680742 [Yanjie Gao] Update SqlParser.scala a7193d8 [gaoyanjie] [SPARK-2234][SQL]Spark SQL basicOperators add Except operator #1151 5c8a224 [Yanjie Gao] update the line less than 100c ee066b3 [Yanjie Gao] Update basicOperators.scala 32a80ab [Yanjie Gao] remove except in HiveQl cf232eb [Yanjie Gao] update 1comment 2space3 left.out f1ea3f3 [Yanjie Gao] remove comment 7ea9b91 [Yanjie Gao] remove annotation 7f3d613 [Yanjie Gao] update .map(_.copy()) 670a1bb [Yanjie Gao] Update HiveQl.scala 3fe7746 [Yanjie Gao] Update SQLQuerySuite.scala a36eb0a [Yanjie Gao] Update basicOperators.scala 7859e56 [Yanjie Gao] Update SparkStrategies.scala 052346d [Yanjie Gao] Subtract is conflict with Subtract(e1,e2) aab3785 [Yanjie Gao] Update SQLQuerySuite.scala 4bf80b1 [Yanjie Gao] update subtract to except 4bdd520 [Yanjie Gao] Update SqlParser.scala 2d4bfbd [Yanjie Gao] Update SQLQuerySuite.scala 0808921 [Yanjie Gao] SQLQuerySuite a8a1948 [Yanjie Gao] SparkStrategies 1fe96c0 [Yanjie Gao] HiveQl.scala update 3305e40 [Yanjie Gao] SqlParser 7a98c37 [Yanjie Gao] Update basicOperators.scala cf5b9d0 [Yanjie Gao] Update basicOperators.scala 8945835 [Yanjie Gao] object SkewJoin extends Strategy 2b98962 [Yanjie Gao] Update SqlParser.scala dd32980 [Yanjie Gao] update1 68815b2 [Yanjie Gao] Reformat the code style 4eb43ec [Yanjie Gao] Update basicOperators.scala aa06072 [Yanjie Gao] Reformat the code sytle --- .../org/apache/spark/sql/catalyst/SqlParser.scala | 3 +++ .../catalyst/plans/logical/basicOperators.scala | 6 ++++++ .../spark/sql/execution/SparkStrategies.scala | 2 ++ .../spark/sql/execution/basicOperators.scala | 15 +++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 14 ++++++++++++++ 5 files changed, 40 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 61762fa2a7c30..ecb11129557bd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -118,6 +118,8 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val UNCACHE = Keyword("UNCACHE") protected val UNION = Keyword("UNION") protected val WHERE = Keyword("WHERE") + protected val EXCEPT = Keyword("EXCEPT") + // Use reflection to find the reserved words defined in this class. protected val reservedWords = @@ -138,6 +140,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected lazy val query: Parser[LogicalPlan] = ( select * ( UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } | + EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} | UNION ~ opt(DISTINCT) ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } ) | insert | cache diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 3e0639867b278..bac5a724647f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -89,6 +89,12 @@ case class Join( } } +case class Except(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { + def output = left.output + + def references = Set.empty +} + case class InsertIntoTable( table: LogicalPlan, partition: Map[String, Option[String]], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 0925605b7c4d9..9e036e127bd56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -273,6 +273,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Limit(limit, planLater(child))(sqlContext) :: Nil case Unions(unionChildren) => execution.Union(unionChildren.map(planLater))(sqlContext) :: Nil + case logical.Except(left,right) => + execution.Except(planLater(left),planLater(right)) :: Nil case logical.Generate(generator, join, outer, _, child) => execution.Generate(generator, join = join, outer = outer, planLater(child)) :: Nil case logical.NoRelation => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index a278f1ca98476..4b59e0b4e58e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -205,3 +205,18 @@ object ExistingRdd { case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { override def execute() = rdd } + +/** + * :: DeveloperApi :: + * Returns a table with the elements from left that are not in right using + * the built-in spark subtract function. + */ +@DeveloperApi +case class Except(left: SparkPlan, right: SparkPlan) extends BinaryNode { + override def output = left.output + + override def execute() = { + left.execute().map(_.copy()).subtract(right.execute().map(_.copy())) + } +} + 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 2c1cb1867010c..5c6701e203d17 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 @@ -371,6 +371,20 @@ class SQLQuerySuite extends QueryTest { (3, null))) } + test("EXCEPT") { + + checkAnswer( + sql("SELECT * FROM lowerCaseData EXCEPT SELECT * FROM upperCaseData "), + (1, "a") :: + (2, "b") :: + (3, "c") :: + (4, "d") :: Nil) + checkAnswer( + sql("SELECT * FROM lowerCaseData EXCEPT SELECT * FROM lowerCaseData "), Nil) + checkAnswer( + sql("SELECT * FROM upperCaseData EXCEPT SELECT * FROM upperCaseData "), Nil) + } + test("SET commands semantics using sql()") { TestSQLContext.settings.synchronized { clear() From fc716589384f70096ba2fe1eb6a4131c61b21c7c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 4 Jul 2014 10:01:19 -0700 Subject: [PATCH 100/124] HOTFIX: Clean before building docs during release. If the docs are built after a Maven build has finished the intermediate state somehow causes a compiler bug during sbt compilation. This just does a clean before attempting to build the docs. --- dev/create-release/create-release.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index c96afe822c6a9..49bf78f60763a 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -125,6 +125,7 @@ scp spark-* \ # Docs cd spark +sbt/sbt clean cd docs PRODUCTION=1 jekyll build echo "Copying release documentation" From 0db5d5a22ef8dd6e3fbb2759cd25aac46bdf2ce9 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 4 Jul 2014 17:33:07 -0700 Subject: [PATCH 101/124] Added SignalLogger to HistoryServer. This was omitted in #1260. @aarondav Author: Reynold Xin Closes #1300 from rxin/historyServer and squashes the following commits: af720a3 [Reynold Xin] Added SignalLogger to HistoryServer. --- .../org/apache/spark/deploy/history/HistoryServer.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 29a78a56c8ed5..56b38ddfc9313 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -27,7 +27,7 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.ui.{WebUI, SparkUI, UIUtils} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{SignalLogger, Utils} /** * A web server that renders SparkUIs of completed applications. @@ -169,10 +169,11 @@ class HistoryServer( * * This launches the HistoryServer as a Spark daemon. */ -object HistoryServer { +object HistoryServer extends Logging { private val conf = new SparkConf def main(argStrings: Array[String]) { + SignalLogger.register(log) initSecurity() val args = new HistoryServerArguments(conf, argStrings) val securityManager = new SecurityManager(conf) From 9d006c97371ddf357e0b821d5c6d1535d9b6fe41 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 4 Jul 2014 19:15:48 -0700 Subject: [PATCH 102/124] [SPARK-2370][SQL] Decrease metadata retrieved for partitioned hive queries. Author: Michael Armbrust Closes #1305 from marmbrus/usePrunerPartitions and squashes the following commits: 744aa20 [Michael Armbrust] Use getAllPartitionsForPruner instead of getPartitions, which avoids retrieving auth data --- .../scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 90eacf4268780..7c24b5cabf614 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -53,7 +53,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val table = client.getTable(databaseName, tableName) val partitions: Seq[Partition] = if (table.isPartitioned) { - client.getPartitions(table) + client.getAllPartitionsForPruner(table).toSeq } else { Nil } From 42f3abd529e84f3b26386abe2bde30666e74b64e Mon Sep 17 00:00:00 2001 From: "ankit.bhardwaj" Date: Fri, 4 Jul 2014 22:06:10 -0700 Subject: [PATCH 103/124] [SPARK-2306]:BoundedPriorityQueue is private and not registered with Kry... Due to the non registration of BoundedPriorityQueue with kryoserializer, operations which are dependend on BoundedPriorityQueue are giving exceptions.One such instance is using top along with kryo serialization. Fixed the issue by registering BoundedPriorityQueue with kryoserializer. Author: ankit.bhardwaj Closes #1299 from AnkitBhardwaj12/BoundedPriorityQueueWithKryoIssue and squashes the following commits: a4ae8ed [ankit.bhardwaj] [SPARK-2306]:BoundedPriorityQueue is private and not registered with Kryo --- .../scala/org/apache/spark/serializer/KryoSerializer.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 82b62aaf61521..1ce4243194798 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -30,6 +30,7 @@ import org.apache.spark.broadcast.HttpBroadcast import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage._ import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} +import org.apache.spark.util.BoundedPriorityQueue import scala.reflect.ClassTag @@ -183,7 +184,8 @@ private[serializer] object KryoSerializer { classOf[GetBlock], classOf[MapStatus], classOf[BlockManagerId], - classOf[Array[Byte]] + classOf[Array[Byte]], + classOf[BoundedPriorityQueue[_]] ) } From 3da8df939ec63064692ba64d9188aeea908b305c Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Sat, 5 Jul 2014 11:48:08 -0700 Subject: [PATCH 104/124] [SPARK-2366] [SQL] Add column pruning for the right side of LeftSemi join. The right side of `LeftSemi` join needs columns only used in join condition. Author: Takuya UESHIN Closes #1301 from ueshin/issues/SPARK-2366 and squashes the following commits: 7677a39 [Takuya UESHIN] Update comments. 786d3a0 [Takuya UESHIN] Rename method name. e0957b1 [Takuya UESHIN] Add column pruning for the right side of LeftSemi join. --- .../sql/catalyst/optimizer/Optimizer.scala | 28 +++++++++++++------ 1 file changed, 20 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index fb517e40677ec..48ca31e70fe7c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -52,6 +52,7 @@ object Optimizer extends RuleExecutor[LogicalPlan] { * - Inserting Projections beneath the following operators: * - Aggregate * - Project <- Join + * - LeftSemiJoin * - Collapse adjacent projections, performing alias substitution. */ object ColumnPruning extends Rule[LogicalPlan] { @@ -62,19 +63,22 @@ object ColumnPruning extends Rule[LogicalPlan] { // Eliminate unneeded attributes from either side of a Join. case Project(projectList, Join(left, right, joinType, condition)) => - // Collect the list of off references required either above or to evaluate the condition. + // Collect the list of all references required either above or to evaluate the condition. val allReferences: Set[Attribute] = projectList.flatMap(_.references).toSet ++ condition.map(_.references).getOrElse(Set.empty) /** Applies a projection only when the child is producing unnecessary attributes */ - def prunedChild(c: LogicalPlan) = - if ((c.outputSet -- allReferences.filter(c.outputSet.contains)).nonEmpty) { - Project(allReferences.filter(c.outputSet.contains).toSeq, c) - } else { - c - } + def pruneJoinChild(c: LogicalPlan) = prunedChild(c, allReferences) - Project(projectList, Join(prunedChild(left), prunedChild(right), joinType, condition)) + Project(projectList, Join(pruneJoinChild(left), pruneJoinChild(right), joinType, condition)) + + // Eliminate unneeded attributes from right side of a LeftSemiJoin. + case Join(left, right, LeftSemi, condition) => + // Collect the list of all references required to evaluate the condition. + val allReferences: Set[Attribute] = + condition.map(_.references).getOrElse(Set.empty) + + Join(left, prunedChild(right, allReferences), LeftSemi, condition) // Combine adjacent Projects. case Project(projectList1, Project(projectList2, child)) => @@ -97,6 +101,14 @@ object ColumnPruning extends Rule[LogicalPlan] { // Eliminate no-op Projects case Project(projectList, child) if child.output == projectList => child } + + /** Applies a projection only when the child is producing unnecessary attributes */ + private def prunedChild(c: LogicalPlan, allReferences: Set[Attribute]) = + if ((c.outputSet -- allReferences.filter(c.outputSet.contains)).nonEmpty) { + Project(allReferences.filter(c.outputSet.contains).toSeq, c) + } else { + c + } } /** From 9d5ecf8205b924dc8a3c13fed68beb78cc5c7553 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Sat, 5 Jul 2014 11:51:48 -0700 Subject: [PATCH 105/124] [SPARK-2327] [SQL] Fix nullabilities of Join/Generate/Aggregate. Fix nullabilities of `Join`/`Generate`/`Aggregate` because: - Output attributes of opposite side of `OuterJoin` should be nullable. - Output attributes of generater side of `Generate` should be nullable if `join` is `true` and `outer` is `true`. - `AttributeReference` of `computedAggregates` of `Aggregate` should be the same as `aggregateExpression`'s. Author: Takuya UESHIN Closes #1266 from ueshin/issues/SPARK-2327 and squashes the following commits: 3ace83a [Takuya UESHIN] Add withNullability to Attribute and use it to change nullabilities. df1ae53 [Takuya UESHIN] Modify nullabilize to leave attribute if not resolved. 799ce56 [Takuya UESHIN] Add nullabilization to Generate of SparkPlan. a0fc9bc [Takuya UESHIN] Fix scalastyle errors. 0e31e37 [Takuya UESHIN] Fix Aggregate resultAttribute nullabilities. 09532ec [Takuya UESHIN] Fix Generate output nullabilities. f20f196 [Takuya UESHIN] Fix Join output nullabilities. --- .../sql/catalyst/analysis/unresolved.scala | 2 ++ .../catalyst/expressions/BoundAttribute.scala | 16 +++++----- .../expressions/namedExpressions.scala | 3 +- .../plans/logical/basicOperators.scala | 31 ++++++++++++++----- .../spark/sql/execution/Aggregate.scala | 4 +-- .../apache/spark/sql/execution/Generate.scala | 12 +++++-- .../apache/spark/sql/execution/joins.scala | 13 +++++++- 7 files changed, 60 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index d629172a7426e..7abeb032964e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -52,6 +52,7 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo override lazy val resolved = false override def newInstance = this + override def withNullability(newNullability: Boolean) = this override def withQualifiers(newQualifiers: Seq[String]) = this // Unresolved attributes are transient at compile time and don't get evaluated during execution. @@ -95,6 +96,7 @@ case class Star( override lazy val resolved = false override def newInstance = this + override def withNullability(newNullability: Boolean) = this override def withQualifiers(newQualifiers: Seq[String]) = this def expand(input: Seq[Attribute]): Seq[NamedExpression] = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 655d4a08fe93b..9ce1f01056462 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -33,14 +33,16 @@ case class BoundReference(ordinal: Int, baseReference: Attribute) type EvaluatedType = Any - def nullable = baseReference.nullable - def dataType = baseReference.dataType - def exprId = baseReference.exprId - def qualifiers = baseReference.qualifiers - def name = baseReference.name + override def nullable = baseReference.nullable + override def dataType = baseReference.dataType + override def exprId = baseReference.exprId + override def qualifiers = baseReference.qualifiers + override def name = baseReference.name - def newInstance = BoundReference(ordinal, baseReference.newInstance) - def withQualifiers(newQualifiers: Seq[String]) = + override def newInstance = BoundReference(ordinal, baseReference.newInstance) + override def withNullability(newNullability: Boolean) = + BoundReference(ordinal, baseReference.withNullability(newNullability)) + override def withQualifiers(newQualifiers: Seq[String]) = BoundReference(ordinal, baseReference.withQualifiers(newQualifiers)) override def toString = s"$baseReference:$ordinal" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 66ae22e95b60e..934bad8c27294 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -57,6 +57,7 @@ abstract class NamedExpression extends Expression { abstract class Attribute extends NamedExpression { self: Product => + def withNullability(newNullability: Boolean): Attribute def withQualifiers(newQualifiers: Seq[String]): Attribute def toAttribute = this @@ -133,7 +134,7 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea /** * Returns a copy of this [[AttributeReference]] with changed nullability. */ - def withNullability(newNullability: Boolean) = { + override def withNullability(newNullability: Boolean) = { if (nullable == newNullability) { this } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index bac5a724647f8..0728fa73fb108 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.{LeftSemi, JoinType} +import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.types._ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { @@ -46,10 +46,16 @@ case class Generate( child: LogicalPlan) extends UnaryNode { - protected def generatorOutput: Seq[Attribute] = - alias + protected def generatorOutput: Seq[Attribute] = { + val output = alias .map(a => generator.output.map(_.withQualifiers(a :: Nil))) .getOrElse(generator.output) + if (join && outer) { + output.map(_.withNullability(true)) + } else { + output + } + } override def output = if (join) child.output ++ generatorOutput else generatorOutput @@ -81,11 +87,20 @@ case class Join( condition: Option[Expression]) extends BinaryNode { override def references = condition.map(_.references).getOrElse(Set.empty) - override def output = joinType match { - case LeftSemi => - left.output - case _ => - left.output ++ right.output + + override def output = { + joinType match { + case LeftSemi => + left.output + case LeftOuter => + left.output ++ right.output.map(_.withNullability(true)) + case RightOuter => + left.output.map(_.withNullability(true)) ++ right.output + case FullOuter => + left.output.map(_.withNullability(true)) ++ right.output.map(_.withNullability(true)) + case _ => + left.output ++ right.output + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala index d85d2d7844e0b..c1ced8bfa404a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala @@ -83,8 +83,8 @@ case class Aggregate( case a: AggregateExpression => ComputedAggregate( a, - BindReferences.bindReference(a, childOutput).asInstanceOf[AggregateExpression], - AttributeReference(s"aggResult:$a", a.dataType, nullable = true)()) + BindReferences.bindReference(a, childOutput), + AttributeReference(s"aggResult:$a", a.dataType, a.nullable)()) } }.toArray diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala index da1e08be59e23..47b3d00262dbb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.catalyst.expressions.{Generator, JoinedRow, Literal, Projection} +import org.apache.spark.sql.catalyst.expressions._ /** * :: DeveloperApi :: @@ -39,8 +39,16 @@ case class Generate( child: SparkPlan) extends UnaryNode { + protected def generatorOutput: Seq[Attribute] = { + if (join && outer) { + generator.output.map(_.withNullability(true)) + } else { + generator.output + } + } + override def output = - if (join) child.output ++ generator.output else generator.output + if (join) child.output ++ generatorOutput else generatorOutput override def execute() = { if (join) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index 32c5f26fe8aa0..7d1f11caae838 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -319,7 +319,18 @@ case class BroadcastNestedLoopJoin( override def otherCopyArgs = sqlContext :: Nil - def output = left.output ++ right.output + override def output = { + joinType match { + case LeftOuter => + left.output ++ right.output.map(_.withNullability(true)) + case RightOuter => + left.output.map(_.withNullability(true)) ++ right.output + case FullOuter => + left.output.map(_.withNullability(true)) ++ right.output.map(_.withNullability(true)) + case _ => + left.output ++ right.output + } + } /** The Streamed Relation */ def left = streamed From f7ce1b3b48f0354434456241188c6a5d954852e2 Mon Sep 17 00:00:00 2001 From: Neville Li Date: Mon, 7 Jul 2014 15:06:14 -0700 Subject: [PATCH 106/124] [SPARK-1977][MLLIB] register mutable BitSet in MovieLenseALS Author: Neville Li Closes #1319 from nevillelyh/gh/SPARK-1977 and squashes the following commits: 1f0a355 [Neville Li] [SPARK-1977][MLLIB] register mutable BitSet in MovieLenseALS --- .../scala/org/apache/spark/examples/mllib/MovieLensALS.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala index 28e201d279f41..98aaedb9d7dc9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala @@ -17,6 +17,8 @@ package org.apache.spark.examples.mllib +import scala.collection.mutable + import com.esotericsoftware.kryo.Kryo import org.apache.log4j.{Level, Logger} import scopt.OptionParser @@ -41,6 +43,7 @@ object MovieLensALS { class ALSRegistrator extends KryoRegistrator { override def registerClasses(kryo: Kryo) { kryo.register(classOf[Rating]) + kryo.register(classOf[mutable.BitSet]) } } From c0b4cf097de50eb2c4b0f0e67da53ee92efc1f77 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 7 Jul 2014 17:01:44 -0700 Subject: [PATCH 107/124] [SPARK-2339][SQL] SQL parser in sql-core is case sensitive, but a table alias is converted to lower case when we create Subquery Reported by http://apache-spark-user-list.1001560.n3.nabble.com/Spark-SQL-Join-throws-exception-td8599.html After we get the table from the catalog, because the table has an alias, we will temporarily insert a Subquery. Then, we convert the table alias to lower case no matter if the parser is case sensitive or not. To see the issue ... ``` val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext.createSchemaRDD case class Person(name: String, age: Int) val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt)) people.registerAsTable("people") sqlContext.sql("select PEOPLE.name from people PEOPLE") ``` The plan is ... ``` == Query Plan == Project ['PEOPLE.name] ExistingRdd [name#0,age#1], MapPartitionsRDD[4] at mapPartitions at basicOperators.scala:176 ``` You can find that `PEOPLE.name` is not resolved. This PR introduces three changes. 1. If a table has an alias, the catalog will not lowercase the alias. If a lowercase alias is needed, the analyzer will do the work. 2. A catalog has a new val caseSensitive that indicates if this catalog is case sensitive or not. For example, a SimpleCatalog is case sensitive, but 3. Corresponding unit tests. With this PR, case sensitivity of database names and table names is handled by the catalog. Case sensitivity of other identifiers are handled by the analyzer. JIRA: https://issues.apache.org/jira/browse/SPARK-2339 Author: Yin Huai Closes #1317 from yhuai/SPARK-2339 and squashes the following commits: 12d8006 [Yin Huai] Handling case sensitivity correctly. This patch introduces three changes. 1. If a table has an alias, the catalog will not lowercase the alias. If a lowercase alias is needed, the analyzer will do the work. 2. A catalog has a new val caseSensitive that indicates if this catalog is case sensitive or not. For example, a SimpleCatalog is case sensitive, but 3. Corresponding unit tests. With this patch, case sensitivity of database names and table names is handled by the catalog. Case sensitivity of other identifiers is handled by the analyzer. --- .../spark/sql/catalyst/analysis/Catalog.scala | 55 +++++++++++---- .../sql/catalyst/analysis/AnalysisSuite.scala | 69 ++++++++++++++++--- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 23 ++++--- ...e table-0-5d14d21a239daa42b086cc895215009a | 14 ++++ .../sql/hive/execution/HiveQuerySuite.scala | 16 +++++ 6 files changed, 149 insertions(+), 30 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/case sensitivity: Hive table-0-5d14d21a239daa42b086cc895215009a diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index f30b5d816703a..0d05d9808b407 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -25,6 +25,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery} * An interface for looking up relations by name. Used by an [[Analyzer]]. */ trait Catalog { + + def caseSensitive: Boolean + def lookupRelation( databaseName: Option[String], tableName: String, @@ -35,22 +38,44 @@ trait Catalog { def unregisterTable(databaseName: Option[String], tableName: String): Unit def unregisterAllTables(): Unit + + protected def processDatabaseAndTableName( + databaseName: Option[String], + tableName: String): (Option[String], String) = { + if (!caseSensitive) { + (databaseName.map(_.toLowerCase), tableName.toLowerCase) + } else { + (databaseName, tableName) + } + } + + protected def processDatabaseAndTableName( + databaseName: String, + tableName: String): (String, String) = { + if (!caseSensitive) { + (databaseName.toLowerCase, tableName.toLowerCase) + } else { + (databaseName, tableName) + } + } } -class SimpleCatalog extends Catalog { +class SimpleCatalog(val caseSensitive: Boolean) extends Catalog { val tables = new mutable.HashMap[String, LogicalPlan]() override def registerTable( databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit = { - tables += ((tableName, plan)) + val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) + tables += ((tblName, plan)) } override def unregisterTable( databaseName: Option[String], tableName: String) = { - tables -= tableName + val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) + tables -= tblName } override def unregisterAllTables() = { @@ -61,12 +86,13 @@ class SimpleCatalog extends Catalog { databaseName: Option[String], tableName: String, alias: Option[String] = None): LogicalPlan = { - val table = tables.get(tableName).getOrElse(sys.error(s"Table Not Found: $tableName")) - val tableWithQualifiers = Subquery(tableName, table) + val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) + val table = tables.get(tblName).getOrElse(sys.error(s"Table Not Found: $tableName")) + val tableWithQualifiers = Subquery(tblName, table) // If an alias was specified by the lookup, wrap the plan in a subquery so that attributes are // properly qualified with this alias. - alias.map(a => Subquery(a.toLowerCase, tableWithQualifiers)).getOrElse(tableWithQualifiers) + alias.map(a => Subquery(a, tableWithQualifiers)).getOrElse(tableWithQualifiers) } } @@ -85,26 +111,28 @@ trait OverrideCatalog extends Catalog { databaseName: Option[String], tableName: String, alias: Option[String] = None): LogicalPlan = { - - val overriddenTable = overrides.get((databaseName, tableName)) + val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) + val overriddenTable = overrides.get((dbName, tblName)) // If an alias was specified by the lookup, wrap the plan in a subquery so that attributes are // properly qualified with this alias. val withAlias = - overriddenTable.map(r => alias.map(a => Subquery(a.toLowerCase, r)).getOrElse(r)) + overriddenTable.map(r => alias.map(a => Subquery(a, r)).getOrElse(r)) - withAlias.getOrElse(super.lookupRelation(databaseName, tableName, alias)) + withAlias.getOrElse(super.lookupRelation(dbName, tblName, alias)) } override def registerTable( databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit = { - overrides.put((databaseName, tableName), plan) + val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) + overrides.put((dbName, tblName), plan) } override def unregisterTable(databaseName: Option[String], tableName: String): Unit = { - overrides.remove((databaseName, tableName)) + val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) + overrides.remove((dbName, tblName)) } override def unregisterAllTables(): Unit = { @@ -117,6 +145,9 @@ trait OverrideCatalog extends Catalog { * relations are already filled in and the analyser needs only to resolve attribute references. */ object EmptyCatalog extends Catalog { + + val caseSensitive: Boolean = true + def lookupRelation( databaseName: Option[String], tableName: String, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index f14df8137683b..0a4fde3de7752 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -17,28 +17,81 @@ package org.apache.spark.sql.catalyst.analysis -import org.scalatest.FunSuite +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.types.IntegerType -/* Implicit conversions */ -import org.apache.spark.sql.catalyst.dsl.expressions._ +class AnalysisSuite extends FunSuite with BeforeAndAfter { + val caseSensitiveCatalog = new SimpleCatalog(true) + val caseInsensitiveCatalog = new SimpleCatalog(false) + val caseSensitiveAnalyze = + new Analyzer(caseSensitiveCatalog, EmptyFunctionRegistry, caseSensitive = true) + val caseInsensitiveAnalyze = + new Analyzer(caseInsensitiveCatalog, EmptyFunctionRegistry, caseSensitive = false) -class AnalysisSuite extends FunSuite { - val analyze = SimpleAnalyzer + val testRelation = LocalRelation(AttributeReference("a", IntegerType, nullable = true)()) - val testRelation = LocalRelation('a.int) + before { + caseSensitiveCatalog.registerTable(None, "TaBlE", testRelation) + caseInsensitiveCatalog.registerTable(None, "TaBlE", testRelation) + } test("analyze project") { assert( - analyze(Project(Seq(UnresolvedAttribute("a")), testRelation)) === + caseSensitiveAnalyze(Project(Seq(UnresolvedAttribute("a")), testRelation)) === + Project(testRelation.output, testRelation)) + + assert( + caseSensitiveAnalyze( + Project(Seq(UnresolvedAttribute("TbL.a")), + UnresolvedRelation(None, "TaBlE", Some("TbL")))) === + Project(testRelation.output, testRelation)) + + val e = intercept[TreeNodeException[_]] { + caseSensitiveAnalyze( + Project(Seq(UnresolvedAttribute("tBl.a")), + UnresolvedRelation(None, "TaBlE", Some("TbL")))) + } + assert(e.getMessage().toLowerCase.contains("unresolved")) + + assert( + caseInsensitiveAnalyze( + Project(Seq(UnresolvedAttribute("TbL.a")), + UnresolvedRelation(None, "TaBlE", Some("TbL")))) === Project(testRelation.output, testRelation)) + + assert( + caseInsensitiveAnalyze( + Project(Seq(UnresolvedAttribute("tBl.a")), + UnresolvedRelation(None, "TaBlE", Some("TbL")))) === + Project(testRelation.output, testRelation)) + } + + test("resolve relations") { + val e = intercept[RuntimeException] { + caseSensitiveAnalyze(UnresolvedRelation(None, "tAbLe", None)) + } + assert(e.getMessage === "Table Not Found: tAbLe") + + assert( + caseSensitiveAnalyze(UnresolvedRelation(None, "TaBlE", None)) === + testRelation) + + assert( + caseInsensitiveAnalyze(UnresolvedRelation(None, "tAbLe", None)) === + testRelation) + + assert( + caseInsensitiveAnalyze(UnresolvedRelation(None, "TaBlE", None)) === + testRelation) } test("throw errors for unresolved attributes during analysis") { val e = intercept[TreeNodeException[_]] { - analyze(Project(Seq(UnresolvedAttribute("abcd")), testRelation)) + caseSensitiveAnalyze(Project(Seq(UnresolvedAttribute("abcd")), testRelation)) } assert(e.getMessage().toLowerCase.contains("unresolved")) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 7edb548678c33..4abd89955bd27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -57,7 +57,7 @@ class SQLContext(@transient val sparkContext: SparkContext) self => @transient - protected[sql] lazy val catalog: Catalog = new SimpleCatalog + protected[sql] lazy val catalog: Catalog = new SimpleCatalog(true) @transient protected[sql] lazy val analyzer: Analyzer = new Analyzer(catalog, EmptyFunctionRegistry, caseSensitive = true) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 7c24b5cabf614..f83068860701f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -45,12 +45,15 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val client = Hive.get(hive.hiveconf) + val caseSensitive: Boolean = false + def lookupRelation( db: Option[String], tableName: String, alias: Option[String]): LogicalPlan = { - val databaseName = db.getOrElse(hive.sessionState.getCurrentDatabase) - val table = client.getTable(databaseName, tableName) + val (dbName, tblName) = processDatabaseAndTableName(db, tableName) + val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) + val table = client.getTable(databaseName, tblName) val partitions: Seq[Partition] = if (table.isPartitioned) { client.getAllPartitionsForPruner(table).toSeq @@ -60,8 +63,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Since HiveQL is case insensitive for table names we make them all lowercase. MetastoreRelation( - databaseName.toLowerCase, - tableName.toLowerCase, + databaseName, + tblName, alias)(table.getTTable, partitions.map(part => part.getTPartition)) } @@ -70,7 +73,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with tableName: String, schema: Seq[Attribute], allowExisting: Boolean = false): Unit = { - val table = new Table(databaseName, tableName) + val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) + val table = new Table(dbName, tblName) val hiveSchema = schema.map(attr => new FieldSchema(attr.name, toMetastoreType(attr.dataType), "")) table.setFields(hiveSchema) @@ -85,7 +89,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with sd.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") sd.setOutputFormat("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat") val serDeInfo = new SerDeInfo() - serDeInfo.setName(tableName) + serDeInfo.setName(tblName) serDeInfo.setSerializationLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") serDeInfo.setParameters(Map[String, String]()) sd.setSerdeInfo(serDeInfo) @@ -104,13 +108,14 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with object CreateTables extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case InsertIntoCreatedTable(db, tableName, child) => - val databaseName = db.getOrElse(hive.sessionState.getCurrentDatabase) + val (dbName, tblName) = processDatabaseAndTableName(db, tableName) + val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) - createTable(databaseName, tableName, child.output) + createTable(databaseName, tblName, child.output) InsertIntoTable( EliminateAnalysisOperators( - lookupRelation(Some(databaseName), tableName, None)), + lookupRelation(Some(databaseName), tblName, None)), Map.empty, child, overwrite = false) diff --git a/sql/hive/src/test/resources/golden/case sensitivity: Hive table-0-5d14d21a239daa42b086cc895215009a b/sql/hive/src/test/resources/golden/case sensitivity: Hive table-0-5d14d21a239daa42b086cc895215009a new file mode 100644 index 0000000000000..4d7127c0faabc --- /dev/null +++ b/sql/hive/src/test/resources/golden/case sensitivity: Hive table-0-5d14d21a239daa42b086cc895215009a @@ -0,0 +1,14 @@ +0 val_0 +4 val_4 +12 val_12 +8 val_8 +0 val_0 +0 val_0 +10 val_10 +5 val_5 +11 val_11 +5 val_5 +2 val_2 +12 val_12 +5 val_5 +9 val_9 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 9f1cd703103ec..a623d29b53973 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 @@ -210,6 +210,22 @@ class HiveQuerySuite extends HiveComparisonTest { } } + createQueryTest("case sensitivity: Hive table", + "SELECT srcalias.KEY, SRCALIAS.value FROM sRc SrCAlias WHERE SrCAlias.kEy < 15") + + test("case sensitivity: registered table") { + val testData: SchemaRDD = + TestHive.sparkContext.parallelize( + TestData(1, "str1") :: + TestData(2, "str2") :: Nil) + testData.registerAsTable("REGisteredTABle") + + assertResult(Array(Array(2, "str2"))) { + hql("SELECT tablealias.A, TABLEALIAS.b FROM reGisteredTABle TableAlias " + + "WHERE TableAliaS.a > 1").collect() + } + } + def isExplanation(result: SchemaRDD) = { val explanation = result.select('plan).collect().map { case Row(plan: String) => plan } explanation.size > 1 && explanation.head.startsWith("Physical execution plan") From 4deeed17c4847f212a4fa1a8685cfe8a12179263 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 7 Jul 2014 17:04:02 -0700 Subject: [PATCH 108/124] [SPARK-2386] [SQL] RowWriteSupport should use the exact types to cast. When execute `saveAsParquetFile` with non-primitive type, `RowWriteSupport` uses wrong type `Int` for `ByteType` and `ShortType`. Author: Takuya UESHIN Closes #1315 from ueshin/issues/SPARK-2386 and squashes the following commits: 20d89ec [Takuya UESHIN] Use None instead of null. bd88741 [Takuya UESHIN] Add a test. 323d1d2 [Takuya UESHIN] Modify RowWriteSupport to use the exact types to cast. --- .../sql/parquet/ParquetTableSupport.scala | 4 +- .../spark/sql/parquet/ParquetQuerySuite.scala | 40 ++++++++++++++++++- 2 files changed, 41 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index bfcbdeb34a92f..9cd5dc5bbd393 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -192,9 +192,9 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { ) ) case IntegerType => writer.addInteger(value.asInstanceOf[Int]) - case ShortType => writer.addInteger(value.asInstanceOf[Int]) + case ShortType => writer.addInteger(value.asInstanceOf[Short]) case LongType => writer.addLong(value.asInstanceOf[Long]) - case ByteType => writer.addInteger(value.asInstanceOf[Int]) + case ByteType => writer.addInteger(value.asInstanceOf[Byte]) case DoubleType => writer.addDouble(value.asInstanceOf[Double]) case FloatType => writer.addFloat(value.asInstanceOf[Float]) case BooleanType => writer.addBoolean(value.asInstanceOf[Boolean]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 2ca0c1cdcbeca..dbf315947ff47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -67,6 +67,19 @@ case class AllDataTypes( byteField: Byte, booleanField: Boolean) +case class AllDataTypesWithNonPrimitiveType( + stringField: String, + intField: Int, + longField: Long, + floatField: Float, + doubleField: Double, + shortField: Short, + byteField: Byte, + booleanField: Boolean, + array: Seq[Int], + map: Map[Int, String], + nested: Nested) + class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { TestData // Load test data tables. @@ -119,6 +132,31 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } } + test("Read/Write All Types with non-primitive type") { + val tempDir = getTempFilePath("parquetTest").getCanonicalPath + val range = (0 to 255) + TestSQLContext.sparkContext.parallelize(range) + .map(x => AllDataTypesWithNonPrimitiveType( + s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0, + Seq(x), Map(x -> s"$x"), Nested(x, s"$x"))) + .saveAsParquetFile(tempDir) + val result = parquetFile(tempDir).collect() + range.foreach { + i => + assert(result(i).getString(0) == s"$i", s"row $i String field did not match, got ${result(i).getString(0)}") + assert(result(i).getInt(1) === i) + assert(result(i).getLong(2) === i.toLong) + assert(result(i).getFloat(3) === i.toFloat) + assert(result(i).getDouble(4) === i.toDouble) + assert(result(i).getShort(5) === i.toShort) + assert(result(i).getByte(6) === i.toByte) + assert(result(i).getBoolean(7) === (i % 2 == 0)) + assert(result(i)(8) === Seq(i)) + assert(result(i)(9) === Map(i -> s"$i")) + assert(result(i)(10) === new GenericRow(Array[Any](i, s"$i"))) + } + } + test("self-join parquet files") { val x = ParquetTestData.testData.as('x) val y = ParquetTestData.testData.as('y) @@ -298,7 +336,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("save and load case class RDD with Nones as parquet") { - val data = OptionalReflectData(null, null, null, null, null) + val data = OptionalReflectData(None, None, None, None, None) val rdd = sparkContext.parallelize(data :: Nil) val file = getTempFilePath("parquet") From f0496ee10847db921a028a34f70385f9b740b3f3 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 7 Jul 2014 17:05:59 -0700 Subject: [PATCH 109/124] [SPARK-2375][SQL] JSON schema inference may not resolve type conflicts correctly for a field inside an array of structs For example, for ``` {"array": [{"field":214748364700}, {"field":1}]} ``` the type of field is resolved as IntType. While, for ``` {"array": [{"field":1}, {"field":214748364700}]} ``` the type of field is resolved as LongType. JIRA: https://issues.apache.org/jira/browse/SPARK-2375 Author: Yin Huai Closes #1308 from yhuai/SPARK-2375 and squashes the following commits: 3e2e312 [Yin Huai] Update unit test. 1b2ff9f [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2375 10794eb [Yin Huai] Correctly resolve the type of a field inside an array of structs. --- .../main/scala/org/apache/spark/sql/json/JsonRDD.scala | 9 +++++---- .../test/scala/org/apache/spark/sql/json/JsonSuite.scala | 8 +++++--- .../scala/org/apache/spark/sql/json/TestJsonData.scala | 3 ++- 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index edf86775579d8..f6cbca96483e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -198,11 +198,12 @@ private[sql] object JsonRDD extends Logging { * in this JSON object can appear in other JSON objects. */ private def allKeysWithValueTypes(m: Map[String, Any]): Set[(String, DataType)] = { - m.map{ + val keyValuePairs = m.map { // Quote the key with backticks to handle cases which have dots // in the field name. - case (key, dataType) => (s"`$key`", dataType) - }.flatMap { + case (key, value) => (s"`$key`", value) + }.toSet + keyValuePairs.flatMap { case (key: String, struct: Map[String, Any]) => { // The value associted with the key is an JSON object. allKeysWithValueTypes(struct).map { @@ -224,7 +225,7 @@ private[sql] object JsonRDD extends Logging { } } case (key: String, value) => (key, typeOfPrimitiveValue(value)) :: Nil - }.toSet + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 10bd9f08f0238..e765cfc83a397 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -451,7 +451,9 @@ class JsonSuite extends QueryTest { val jsonSchemaRDD = jsonRDD(arrayElementTypeConflict) val expectedSchema = - AttributeReference("array", ArrayType(StringType), true)() :: Nil + AttributeReference("array1", ArrayType(StringType), true)() :: + AttributeReference("array2", ArrayType(StructType( + StructField("field", LongType, true) :: Nil)), true)() :: Nil comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) @@ -460,12 +462,12 @@ class JsonSuite extends QueryTest { checkAnswer( sql("select * from jsonTable"), Seq(Seq("1", "1.1", "true", null, "[]", "{}", "[2,3,4]", - """{"field":str}""")) :: Nil + """{"field":str}"""), Seq(Seq(214748364700L), Seq(1))) :: Nil ) // Treat an element as a number. checkAnswer( - sql("select array[0] + 1 from jsonTable"), + sql("select array1[0] + 1 from jsonTable"), 2 ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index 065e04046e8a6..d0180f3754f22 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -72,7 +72,8 @@ object TestJsonData { val arrayElementTypeConflict = TestSQLContext.sparkContext.parallelize( - """{"array": [1, 1.1, true, null, [], {}, [2,3,4], {"field":"str"}]}""" :: Nil) + """{"array1": [1, 1.1, true, null, [], {}, [2,3,4], {"field":"str"}], + "array2": [{"field":214748364700}, {"field":1}]}""" :: Nil) val missingFields = TestSQLContext.sparkContext.parallelize( From 4352a2fdaa64efee7158eabef65703460ff284ec Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 7 Jul 2014 18:37:38 -0700 Subject: [PATCH 110/124] [SPARK-2376][SQL] Selecting list values inside nested JSON objects raises java.lang.IllegalArgumentException JIRA: https://issues.apache.org/jira/browse/SPARK-2376 Author: Yin Huai Closes #1320 from yhuai/SPARK-2376 and squashes the following commits: 0107417 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2376 480803d [Yin Huai] Correctly handling JSON arrays in PySpark. --- python/pyspark/sql.py | 24 +++++----- .../org/apache/spark/sql/SchemaRDD.scala | 45 ++++++++++++------- 2 files changed, 44 insertions(+), 25 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 5051c82da32a7..ffe177576f363 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -152,10 +152,12 @@ def jsonFile(self, path): >>> ofn.close() >>> srdd = sqlCtx.jsonFile(jsonFile) >>> sqlCtx.registerRDDAsTable(srdd, "table1") - >>> srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2, field3 as f3 from table1") - >>> srdd2.collect() == [{"f1": 1, "f2": "row1", "f3":{"field4":11}}, - ... {"f1": 2, "f2": "row2", "f3":{"field4":22}}, - ... {"f1": 3, "f2": "row3", "f3":{"field4":33}}] + >>> srdd2 = sqlCtx.sql( + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, field6 as f4 from table1") + >>> srdd2.collect() == [ + ... {"f1":1, "f2":"row1", "f3":{"field4":11, "field5": None}, "f4":None}, + ... {"f1":2, "f2":None, "f3":{"field4":22, "field5": [10, 11]}, "f4":[{"field7": "row2"}]}, + ... {"f1":None, "f2":"row3", "f3":{"field4":33, "field5": []}, "f4":None}] True """ jschema_rdd = self._ssql_ctx.jsonFile(path) @@ -167,10 +169,12 @@ def jsonRDD(self, rdd): >>> srdd = sqlCtx.jsonRDD(json) >>> sqlCtx.registerRDDAsTable(srdd, "table1") - >>> srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2, field3 as f3 from table1") - >>> srdd2.collect() == [{"f1": 1, "f2": "row1", "f3":{"field4":11}}, - ... {"f1": 2, "f2": "row2", "f3":{"field4":22}}, - ... {"f1": 3, "f2": "row3", "f3":{"field4":33}}] + >>> srdd2 = sqlCtx.sql( + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, field6 as f4 from table1") + >>> srdd2.collect() == [ + ... {"f1":1, "f2":"row1", "f3":{"field4":11, "field5": None}, "f4":None}, + ... {"f1":2, "f2":None, "f3":{"field4":22, "field5": [10, 11]}, "f4":[{"field7": "row2"}]}, + ... {"f1":None, "f2":"row3", "f3":{"field4":33, "field5": []}, "f4":None}] True """ def func(split, iterator): @@ -492,8 +496,8 @@ def _test(): globs['rdd'] = sc.parallelize([{"field1" : 1, "field2" : "row1"}, {"field1" : 2, "field2": "row2"}, {"field1" : 3, "field2": "row3"}]) jsonStrings = ['{"field1": 1, "field2": "row1", "field3":{"field4":11}}', - '{"field1" : 2, "field2": "row2", "field3":{"field4":22}}', - '{"field1" : 3, "field2": "row3", "field3":{"field4":33}}'] + '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]}, "field6":[{"field7": "row2"}]}', + '{"field1" : null, "field2": "row3", "field3":{"field4":33, "field5": []}}'] globs['jsonStrings'] = jsonStrings globs['json'] = sc.parallelize(jsonStrings) globs['nestedRdd1'] = sc.parallelize([ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 8f9f54f610e9d..8bcfc7c064c2f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -17,6 +17,11 @@ package org.apache.spark.sql +import java.util.{Map => JMap, List => JList, Set => JSet} + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ + import net.razorvine.pickle.Pickler import org.apache.spark.{Dependency, OneToOneDependency, Partition, Partitioner, TaskContext} @@ -27,10 +32,9 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} -import org.apache.spark.sql.catalyst.types.{DataType, StructType, BooleanType} +import org.apache.spark.sql.catalyst.types.{ArrayType, BooleanType, StructType} import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} import org.apache.spark.api.java.JavaRDD -import java.util.{Map => JMap} /** * :: AlphaComponent :: @@ -359,6 +363,28 @@ class SchemaRDD( case (obj, (name, dataType)) => dataType match { case struct: StructType => map.put(name, rowToMap(obj.asInstanceOf[Row], struct)) + case array @ ArrayType(struct: StructType) => + val arrayValues = obj match { + case seq: Seq[Any] => + seq.map(element => rowToMap(element.asInstanceOf[Row], struct)).asJava + case list: JList[Any] => + list.map(element => rowToMap(element.asInstanceOf[Row], struct)) + case set: JSet[Any] => + set.map(element => rowToMap(element.asInstanceOf[Row], struct)) + case array if array != null && array.getClass.isArray => + array.asInstanceOf[Array[Any]].map { + element => rowToMap(element.asInstanceOf[Row], struct) + } + case other => other + } + map.put(name, arrayValues) + case array: ArrayType => { + val arrayValues = obj match { + case seq: Seq[Any] => seq.asJava + case other => other + } + map.put(name, arrayValues) + } case other => map.put(name, obj) } } @@ -366,22 +392,11 @@ class SchemaRDD( map } - // TODO: Actually, the schema of a row should be represented by a StructType instead of - // a Seq[Attribute]. Once we have finished that change, we can just use rowToMap to - // construct the Map for python. - val fields: Seq[(String, DataType)] = this.queryExecution.analyzed.output.map( - field => (field.name, field.dataType)) + val rowSchema = StructType.fromAttributes(this.queryExecution.analyzed.output) this.mapPartitions { iter => val pickle = new Pickler iter.map { row => - val map: JMap[String, Any] = new java.util.HashMap - row.zip(fields).foreach { case (obj, (name, dataType)) => - dataType match { - case struct: StructType => map.put(name, rowToMap(obj.asInstanceOf[Row], struct)) - case other => map.put(name, obj) - } - } - map + rowToMap(row, rowSchema) }.grouped(10).map(batched => pickle.dumps(batched.toArray)) } } From 50561f4396be7d641feb2a7a54a374d294628231 Mon Sep 17 00:00:00 2001 From: Yanjie Gao Date: Mon, 7 Jul 2014 19:40:04 -0700 Subject: [PATCH 111/124] [SPARK-2235][SQL]Spark SQL basicOperator add Intersect operator Hi all, I want to submit a basic operator Intersect For example , in sql case select * from table1 intersect select * from table2 So ,i want use this operator support this function in Spark SQL This operator will return the the intersection of SparkPlan child table RDD . JIRA:https://issues.apache.org/jira/browse/SPARK-2235 Author: Yanjie Gao Author: YanjieGao <396154235@qq.com> Closes #1150 from YanjieGao/patch-5 and squashes the following commits: 4629afe [YanjieGao] reformat the code bdc2ac0 [YanjieGao] reformat the code as Michael's suggestion 3b29ad6 [YanjieGao] Merge remote branch 'upstream/master' into patch-5 1cfbfe6 [YanjieGao] refomat some files ea78f33 [YanjieGao] resolve conflict and add annotation on basicOperator and remove HiveQl 0c7cca5 [YanjieGao] modify format problem a802ca8 [YanjieGao] Merge remote branch 'upstream/master' into patch-5 5e374c7 [YanjieGao] resolve conflict in SparkStrategies and basicOperator f7961f6 [Yanjie Gao] update the line less than bdc4a05 [Yanjie Gao] Update basicOperators.scala 0b49837 [Yanjie Gao] delete the annotation f1288b4 [Yanjie Gao] delete annotation e2b64be [Yanjie Gao] Update basicOperators.scala 4dd453e [Yanjie Gao] Update SQLQuerySuite.scala 790765d [Yanjie Gao] Update SparkStrategies.scala ac73e60 [Yanjie Gao] Update basicOperators.scala d4ac5e5 [Yanjie Gao] Update HiveQl.scala 61e88e7 [Yanjie Gao] Update SqlParser.scala 469f099 [Yanjie Gao] Update basicOperators.scala e5bff61 [Yanjie Gao] Spark SQL basicOperator add Intersect operator --- .../org/apache/spark/sql/catalyst/SqlParser.scala | 2 ++ .../sql/catalyst/plans/logical/basicOperators.scala | 5 +++++ .../spark/sql/execution/SparkStrategies.scala | 2 ++ .../apache/spark/sql/execution/basicOperators.scala | 13 +++++++++++++ .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 11 +++++++++++ 5 files changed, 33 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index ecb11129557bd..e5653c5b14ac1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -118,6 +118,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val UNCACHE = Keyword("UNCACHE") protected val UNION = Keyword("UNION") protected val WHERE = Keyword("WHERE") + protected val INTERSECT = Keyword("INTERSECT") protected val EXCEPT = Keyword("EXCEPT") @@ -140,6 +141,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected lazy val query: Parser[LogicalPlan] = ( select * ( UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } | + INTERSECT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Intersect(q1, q2) } | EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} | UNION ~ opt(DISTINCT) ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } ) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 0728fa73fb108..1537de259c5b4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -210,3 +210,8 @@ case class Distinct(child: LogicalPlan) extends UnaryNode { case object NoRelation extends LeafNode { override def output = Nil } + +case class Intersect(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { + override def output = left.output + override def references = Set.empty +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 9e036e127bd56..7080074a69c07 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -275,6 +275,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Union(unionChildren.map(planLater))(sqlContext) :: Nil case logical.Except(left,right) => execution.Except(planLater(left),planLater(right)) :: Nil + case logical.Intersect(left, right) => + execution.Intersect(planLater(left), planLater(right)) :: Nil case logical.Generate(generator, join, outer, _, child) => execution.Generate(generator, join = join, outer = outer, planLater(child)) :: Nil case logical.NoRelation => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 4b59e0b4e58e4..e8816f0b3cd9c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -220,3 +220,16 @@ case class Except(left: SparkPlan, right: SparkPlan) extends BinaryNode { } } +/** + * :: DeveloperApi :: + * Returns the rows in left that also appear in right using the built in spark + * intersection function. + */ +@DeveloperApi +case class Intersect(left: SparkPlan, right: SparkPlan) extends BinaryNode { + override def output = children.head.output + + override def execute() = { + left.execute().map(_.copy()).intersection(right.execute().map(_.copy())) + } +} 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 5c6701e203d17..fa1f32f8a49a9 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 @@ -385,6 +385,17 @@ class SQLQuerySuite extends QueryTest { sql("SELECT * FROM upperCaseData EXCEPT SELECT * FROM upperCaseData "), Nil) } + test("INTERSECT") { + checkAnswer( + sql("SELECT * FROM lowerCaseData INTERSECT SELECT * FROM lowerCaseData"), + (1, "a") :: + (2, "b") :: + (3, "c") :: + (4, "d") :: Nil) + checkAnswer( + sql("SELECT * FROM lowerCaseData INTERSECT SELECT * FROM upperCaseData"), Nil) + } + test("SET commands semantics using sql()") { TestSQLContext.settings.synchronized { clear() From 0128905eea9f8c597ca238b14c18908995511e76 Mon Sep 17 00:00:00 2001 From: Rishi Verma Date: Tue, 8 Jul 2014 00:29:23 -0700 Subject: [PATCH 112/124] Updated programming-guide.md Made sure that readers know the random number generator seed argument, within the 'takeSample' method, is optional. Author: Rishi Verma Closes #1324 from riverma/patch-1 and squashes the following commits: 4699676 [Rishi Verma] Updated programming-guide.md --- docs/programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 06e4c4ce527e1..b09d6347cd1b2 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -975,8 +975,8 @@ for details.
    - - + + From 3cd5029be709307415f911236472a685e406e763 Mon Sep 17 00:00:00 2001 From: witgo Date: Tue, 8 Jul 2014 00:31:42 -0700 Subject: [PATCH 113/124] =?UTF-8?q?Resolve=20sbt=20warnings=20during=20bui?= =?UTF-8?q?ld=20=E2=85=A1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Author: witgo Closes #1153 from witgo/expectResult and squashes the following commits: 97541d8 [witgo] merge master ead26e7 [witgo] Resolve sbt warnings during build --- .../org/apache/spark/AccumulatorSuite.scala | 6 +- .../apache/spark/util/NextIteratorSuite.scala | 38 +++++----- .../spark/util/SizeEstimatorSuite.scala | 70 +++++++++---------- .../spark/sql/columnar/ColumnStatsSuite.scala | 6 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 8 +-- .../columnar/NullableColumnBuilderSuite.scala | 14 ++-- .../compression/BooleanBitSetSuite.scala | 10 +-- .../compression/DictionaryEncodingSuite.scala | 10 +-- .../compression/IntegralDeltaSuite.scala | 16 ++--- .../compression/RunLengthEncodingSuite.scala | 10 +-- 10 files changed, 94 insertions(+), 94 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 3aab88e9e9196..52d1d5277658e 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -61,7 +61,7 @@ class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext { val acc : Accumulator[Int] = sc.accumulator(0) val d = sc.parallelize(1 to 20) - evaluating {d.foreach{x => acc.value = x}} should produce [Exception] + an [Exception] should be thrownBy {d.foreach{x => acc.value = x}} } test ("add value to collection accumulators") { @@ -87,11 +87,11 @@ class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext { sc = new SparkContext("local[" + nThreads + "]", "test") val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) val d = sc.parallelize(1 to maxI) - evaluating { + an [SparkException] should be thrownBy { d.foreach { x => acc.value += x } - } should produce [SparkException] + } resetSparkContext() } } diff --git a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala index cf438a3d72a06..72e81f3f1a884 100644 --- a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala @@ -27,45 +27,45 @@ import org.scalatest.Matchers class NextIteratorSuite extends FunSuite with Matchers { test("one iteration") { val i = new StubIterator(Buffer(1)) - i.hasNext should be === true - i.next should be === 1 - i.hasNext should be === false + i.hasNext should be (true) + i.next should be (1) + i.hasNext should be (false) intercept[NoSuchElementException] { i.next() } } test("two iterations") { val i = new StubIterator(Buffer(1, 2)) - i.hasNext should be === true - i.next should be === 1 - i.hasNext should be === true - i.next should be === 2 - i.hasNext should be === false + i.hasNext should be (true) + i.next should be (1) + i.hasNext should be (true) + i.next should be (2) + i.hasNext should be (false) intercept[NoSuchElementException] { i.next() } } test("empty iteration") { val i = new StubIterator(Buffer()) - i.hasNext should be === false + i.hasNext should be (false) intercept[NoSuchElementException] { i.next() } } test("close is called once for empty iterations") { val i = new StubIterator(Buffer()) - i.hasNext should be === false - i.hasNext should be === false - i.closeCalled should be === 1 + i.hasNext should be (false) + i.hasNext should be (false) + i.closeCalled should be (1) } test("close is called once for non-empty iterations") { val i = new StubIterator(Buffer(1, 2)) - i.next should be === 1 - i.next should be === 2 + i.next should be (1) + i.next should be (2) // close isn't called until we check for the next element - i.closeCalled should be === 0 - i.hasNext should be === false - i.closeCalled should be === 1 - i.hasNext should be === false - i.closeCalled should be === 1 + i.closeCalled should be (0) + i.hasNext should be (false) + i.closeCalled should be (1) + i.hasNext should be (false) + i.closeCalled should be (1) } class StubIterator(ints: Buffer[Int]) extends NextIterator[Int] { diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index b583a8bd46ae5..f9d1af88f3a13 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -63,53 +63,53 @@ class SizeEstimatorSuite } test("simple classes") { - expectResult(16)(SizeEstimator.estimate(new DummyClass1)) - expectResult(16)(SizeEstimator.estimate(new DummyClass2)) - expectResult(24)(SizeEstimator.estimate(new DummyClass3)) - expectResult(24)(SizeEstimator.estimate(new DummyClass4(null))) - expectResult(48)(SizeEstimator.estimate(new DummyClass4(new DummyClass3))) + assertResult(16)(SizeEstimator.estimate(new DummyClass1)) + assertResult(16)(SizeEstimator.estimate(new DummyClass2)) + assertResult(24)(SizeEstimator.estimate(new DummyClass3)) + assertResult(24)(SizeEstimator.estimate(new DummyClass4(null))) + assertResult(48)(SizeEstimator.estimate(new DummyClass4(new DummyClass3))) } // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors // (Sun vs IBM). Use a DummyString class to make tests deterministic. test("strings") { - expectResult(40)(SizeEstimator.estimate(DummyString(""))) - expectResult(48)(SizeEstimator.estimate(DummyString("a"))) - expectResult(48)(SizeEstimator.estimate(DummyString("ab"))) - expectResult(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) + assertResult(40)(SizeEstimator.estimate(DummyString(""))) + assertResult(48)(SizeEstimator.estimate(DummyString("a"))) + assertResult(48)(SizeEstimator.estimate(DummyString("ab"))) + assertResult(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) } test("primitive arrays") { - expectResult(32)(SizeEstimator.estimate(new Array[Byte](10))) - expectResult(40)(SizeEstimator.estimate(new Array[Char](10))) - expectResult(40)(SizeEstimator.estimate(new Array[Short](10))) - expectResult(56)(SizeEstimator.estimate(new Array[Int](10))) - expectResult(96)(SizeEstimator.estimate(new Array[Long](10))) - expectResult(56)(SizeEstimator.estimate(new Array[Float](10))) - expectResult(96)(SizeEstimator.estimate(new Array[Double](10))) - expectResult(4016)(SizeEstimator.estimate(new Array[Int](1000))) - expectResult(8016)(SizeEstimator.estimate(new Array[Long](1000))) + assertResult(32)(SizeEstimator.estimate(new Array[Byte](10))) + assertResult(40)(SizeEstimator.estimate(new Array[Char](10))) + assertResult(40)(SizeEstimator.estimate(new Array[Short](10))) + assertResult(56)(SizeEstimator.estimate(new Array[Int](10))) + assertResult(96)(SizeEstimator.estimate(new Array[Long](10))) + assertResult(56)(SizeEstimator.estimate(new Array[Float](10))) + assertResult(96)(SizeEstimator.estimate(new Array[Double](10))) + assertResult(4016)(SizeEstimator.estimate(new Array[Int](1000))) + assertResult(8016)(SizeEstimator.estimate(new Array[Long](1000))) } test("object arrays") { // Arrays containing nulls should just have one pointer per element - expectResult(56)(SizeEstimator.estimate(new Array[String](10))) - expectResult(56)(SizeEstimator.estimate(new Array[AnyRef](10))) + assertResult(56)(SizeEstimator.estimate(new Array[String](10))) + assertResult(56)(SizeEstimator.estimate(new Array[AnyRef](10))) // For object arrays with non-null elements, each object should take one pointer plus // however many bytes that class takes. (Note that Array.fill calls the code in its // second parameter separately for each object, so we get distinct objects.) - expectResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass1))) - expectResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass2))) - expectResult(296)(SizeEstimator.estimate(Array.fill(10)(new DummyClass3))) - expectResult(56)(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2))) + assertResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass1))) + assertResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass2))) + assertResult(296)(SizeEstimator.estimate(Array.fill(10)(new DummyClass3))) + assertResult(56)(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2))) // Past size 100, our samples 100 elements, but we should still get the right size. - expectResult(28016)(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3))) + assertResult(28016)(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3))) // If an array contains the *same* element many times, we should only count it once. val d1 = new DummyClass1 - expectResult(72)(SizeEstimator.estimate(Array.fill(10)(d1))) // 10 pointers plus 8-byte object - expectResult(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // 100 pointers plus 8-byte object + assertResult(72)(SizeEstimator.estimate(Array.fill(10)(d1))) // 10 pointers plus 8-byte object + assertResult(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // 100 pointers plus 8-byte object // Same thing with huge array containing the same element many times. Note that this won't // return exactly 4032 because it can't tell that *all* the elements will equal the first @@ -127,10 +127,10 @@ class SizeEstimatorSuite val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() - expectResult(40)(SizeEstimator.estimate(DummyString(""))) - expectResult(48)(SizeEstimator.estimate(DummyString("a"))) - expectResult(48)(SizeEstimator.estimate(DummyString("ab"))) - expectResult(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) + assertResult(40)(SizeEstimator.estimate(DummyString(""))) + assertResult(48)(SizeEstimator.estimate(DummyString("a"))) + assertResult(48)(SizeEstimator.estimate(DummyString("ab"))) + assertResult(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) resetOrClear("os.arch", arch) } @@ -142,10 +142,10 @@ class SizeEstimatorSuite val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() - expectResult(56)(SizeEstimator.estimate(DummyString(""))) - expectResult(64)(SizeEstimator.estimate(DummyString("a"))) - expectResult(64)(SizeEstimator.estimate(DummyString("ab"))) - expectResult(72)(SizeEstimator.estimate(DummyString("abcdefgh"))) + assertResult(56)(SizeEstimator.estimate(DummyString(""))) + assertResult(64)(SizeEstimator.estimate(DummyString("a"))) + assertResult(64)(SizeEstimator.estimate(DummyString("ab"))) + assertResult(72)(SizeEstimator.estimate(DummyString("abcdefgh"))) resetOrClear("os.arch", arch) resetOrClear("spark.test.useCompressedOops", oops) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index 78640b876d4aa..6f0d46d816266 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -39,7 +39,7 @@ class ColumnStatsSuite extends FunSuite { test(s"$columnStatsName: empty") { val columnStats = columnStatsClass.newInstance() - expectResult(columnStats.initialBounds, "Wrong initial bounds") { + assertResult(columnStats.initialBounds, "Wrong initial bounds") { (columnStats.lowerBound, columnStats.upperBound) } } @@ -54,8 +54,8 @@ class ColumnStatsSuite extends FunSuite { val values = rows.map(_.head.asInstanceOf[T#JvmType]) val ordering = columnType.dataType.ordering.asInstanceOf[Ordering[T#JvmType]] - expectResult(values.min(ordering), "Wrong lower bound")(columnStats.lowerBound) - expectResult(values.max(ordering), "Wrong upper bound")(columnStats.upperBound) + assertResult(values.min(ordering), "Wrong lower bound")(columnStats.lowerBound) + assertResult(values.max(ordering), "Wrong upper bound")(columnStats.upperBound) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 71be41056768f..314b7d317ed75 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -35,7 +35,7 @@ class ColumnTypeSuite extends FunSuite with Logging { BOOLEAN -> 1, STRING -> 8, BINARY -> 16, GENERIC -> 16) checks.foreach { case (columnType, expectedSize) => - expectResult(expectedSize, s"Wrong defaultSize for $columnType") { + assertResult(expectedSize, s"Wrong defaultSize for $columnType") { columnType.defaultSize } } @@ -47,7 +47,7 @@ class ColumnTypeSuite extends FunSuite with Logging { value: JvmType, expected: Int) { - expectResult(expected, s"Wrong actualSize for $columnType") { + assertResult(expected, s"Wrong actualSize for $columnType") { columnType.actualSize(value) } } @@ -127,7 +127,7 @@ class ColumnTypeSuite extends FunSuite with Logging { val length = buffer.getInt() assert(length === serializedObj.length) - expectResult(obj, "Deserialized object didn't equal to the original object") { + assertResult(obj, "Deserialized object didn't equal to the original object") { val bytes = new Array[Byte](length) buffer.get(bytes, 0, length) SparkSqlSerializer.deserialize(bytes) @@ -136,7 +136,7 @@ class ColumnTypeSuite extends FunSuite with Logging { buffer.rewind() buffer.putInt(serializedObj.length).put(serializedObj) - expectResult(obj, "Deserialized object didn't equal to the original object") { + assertResult(obj, "Deserialized object didn't equal to the original object") { buffer.rewind() SparkSqlSerializer.deserialize(GENERIC.extract(buffer)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index d9d1e1bfddb75..d8898527baa39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -48,8 +48,8 @@ class NullableColumnBuilderSuite extends FunSuite { val columnBuilder = TestNullableColumnBuilder(columnType) val buffer = columnBuilder.build() - expectResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) - expectResult(0, "Wrong null count")(buffer.getInt()) + assertResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) + assertResult(0, "Wrong null count")(buffer.getInt()) assert(!buffer.hasRemaining) } @@ -63,8 +63,8 @@ class NullableColumnBuilderSuite extends FunSuite { val buffer = columnBuilder.build() - expectResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) - expectResult(0, "Wrong null count")(buffer.getInt()) + assertResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) + assertResult(0, "Wrong null count")(buffer.getInt()) } test(s"$typeName column builder: null values") { @@ -79,11 +79,11 @@ class NullableColumnBuilderSuite extends FunSuite { val buffer = columnBuilder.build() - expectResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) - expectResult(4, "Wrong null count")(buffer.getInt()) + assertResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) + assertResult(4, "Wrong null count")(buffer.getInt()) // For null positions - (1 to 7 by 2).foreach(expectResult(_, "Wrong null position")(buffer.getInt())) + (1 to 7 by 2).foreach(assertResult(_, "Wrong null position")(buffer.getInt())) // For non-null values (0 until 4).foreach { _ => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala index 93259a19b9fe7..5fba00480967c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala @@ -48,18 +48,18 @@ class BooleanBitSetSuite extends FunSuite { } // 4 extra bytes for compression scheme type ID - expectResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) + assertResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) // Skips column header buffer.position(headerSize) - expectResult(BooleanBitSet.typeId, "Wrong compression scheme ID")(buffer.getInt()) - expectResult(count, "Wrong element count")(buffer.getInt()) + assertResult(BooleanBitSet.typeId, "Wrong compression scheme ID")(buffer.getInt()) + assertResult(count, "Wrong element count")(buffer.getInt()) var word = 0: Long for (i <- 0 until count) { val bit = i % BITS_PER_LONG word = if (bit == 0) buffer.getLong() else word - expectResult(values(i), s"Wrong value in compressed buffer, index=$i") { + assertResult(values(i), s"Wrong value in compressed buffer, index=$i") { (word & ((1: Long) << bit)) != 0 } } @@ -75,7 +75,7 @@ class BooleanBitSetSuite extends FunSuite { if (values.nonEmpty) { values.foreach { assert(decoder.hasNext) - expectResult(_, "Wrong decoded value")(decoder.next()) + assertResult(_, "Wrong decoded value")(decoder.next()) } } assert(!decoder.hasNext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala index 198dcd8819341..d8ae2a26778c9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -71,22 +71,22 @@ class DictionaryEncodingSuite extends FunSuite { // 2 bytes for each `Short` val compressedSize = 4 + dictionarySize + 2 * inputSeq.length // 4 extra bytes for compression scheme type ID - expectResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) + assertResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) // Skips column header buffer.position(headerSize) - expectResult(DictionaryEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) + assertResult(DictionaryEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) val dictionary = buildDictionary(buffer).toMap dictValues.foreach { i => - expectResult(i, "Wrong dictionary entry") { + assertResult(i, "Wrong dictionary entry") { dictionary(values(i)) } } inputSeq.foreach { i => - expectResult(i.toShort, "Wrong column element value")(buffer.getShort()) + assertResult(i.toShort, "Wrong column element value")(buffer.getShort()) } // ------------- @@ -101,7 +101,7 @@ class DictionaryEncodingSuite extends FunSuite { if (inputSeq.nonEmpty) { inputSeq.foreach { i => assert(decoder.hasNext) - expectResult(values(i), "Wrong decoded value")(decoder.next()) + assertResult(values(i), "Wrong decoded value")(decoder.next()) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala index 46af6e001c633..17619dcf974e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -69,21 +69,21 @@ class IntegralDeltaSuite extends FunSuite { }) // 4 extra bytes for compression scheme type ID - expectResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) + assertResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) buffer.position(headerSize) - expectResult(scheme.typeId, "Wrong compression scheme ID")(buffer.getInt()) + assertResult(scheme.typeId, "Wrong compression scheme ID")(buffer.getInt()) if (input.nonEmpty) { - expectResult(Byte.MinValue, "The first byte should be an escaping mark")(buffer.get()) - expectResult(input.head, "The first value is wrong")(columnType.extract(buffer)) + assertResult(Byte.MinValue, "The first byte should be an escaping mark")(buffer.get()) + assertResult(input.head, "The first value is wrong")(columnType.extract(buffer)) (input.tail, deltas).zipped.foreach { (value, delta) => if (math.abs(delta) <= Byte.MaxValue) { - expectResult(delta, "Wrong delta")(buffer.get()) + assertResult(delta, "Wrong delta")(buffer.get()) } else { - expectResult(Byte.MinValue, "Expecting escaping mark here")(buffer.get()) - expectResult(value, "Wrong value")(columnType.extract(buffer)) + assertResult(Byte.MinValue, "Expecting escaping mark here")(buffer.get()) + assertResult(value, "Wrong value")(columnType.extract(buffer)) } } } @@ -99,7 +99,7 @@ class IntegralDeltaSuite extends FunSuite { if (input.nonEmpty) { input.foreach{ assert(decoder.hasNext) - expectResult(_, "Wrong decoded value")(decoder.next()) + assertResult(_, "Wrong decoded value")(decoder.next()) } } assert(!decoder.hasNext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala index d3b73ba19d476..40115beb98899 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -61,15 +61,15 @@ class RunLengthEncodingSuite extends FunSuite { }.sum // 4 extra bytes for compression scheme type ID - expectResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) + assertResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) // Skips column header buffer.position(headerSize) - expectResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) + assertResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) inputRuns.foreach { case (index, run) => - expectResult(values(index), "Wrong column element value")(columnType.extract(buffer)) - expectResult(run, "Wrong run length")(buffer.getInt()) + assertResult(values(index), "Wrong column element value")(columnType.extract(buffer)) + assertResult(run, "Wrong run length")(buffer.getInt()) } // ------------- @@ -84,7 +84,7 @@ class RunLengthEncodingSuite extends FunSuite { if (inputSeq.nonEmpty) { inputSeq.foreach { i => assert(decoder.hasNext) - expectResult(values(i), "Wrong decoded value")(decoder.next()) + assertResult(values(i), "Wrong decoded value")(decoder.next()) } } From 5a4063645dd7bb4cd8bda890785235729804ab09 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 8 Jul 2014 00:41:46 -0700 Subject: [PATCH 114/124] [SPARK-2391][SQL] Custom take() for LIMIT queries. Using Spark's take can result in an entire in-memory partition to be shipped in order to retrieve a single row. Author: Michael Armbrust Closes #1318 from marmbrus/takeLimit and squashes the following commits: 77289a5 [Michael Armbrust] Update scala doc 32f0674 [Michael Armbrust] Custom take implementation for LIMIT queries. --- .../spark/sql/execution/basicOperators.scala | 51 +++++++++++++++++-- 1 file changed, 47 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index e8816f0b3cd9c..97abd636ab5fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution +import scala.collection.mutable.ArrayBuffer import scala.reflect.runtime.universe.TypeTag import org.apache.spark.annotation.DeveloperApi @@ -83,9 +84,9 @@ case class Union(children: Seq[SparkPlan])(@transient sqlContext: SQLContext) ex * :: DeveloperApi :: * Take the first limit elements. Note that the implementation is different depending on whether * this is a terminal operator or not. If it is terminal and is invoked using executeCollect, - * this operator uses Spark's take method on the Spark driver. If it is not terminal or is - * invoked using execute, we first take the limit on each partition, and then repartition all the - * data to a single partition to compute the global limit. + * this operator uses something similar to Spark's take method on the Spark driver. If it is not + * terminal or is invoked using execute, we first take the limit on each partition, and then + * repartition all the data to a single partition to compute the global limit. */ @DeveloperApi case class Limit(limit: Int, child: SparkPlan)(@transient sqlContext: SQLContext) @@ -97,7 +98,49 @@ case class Limit(limit: Int, child: SparkPlan)(@transient sqlContext: SQLContext override def output = child.output - override def executeCollect() = child.execute().map(_.copy()).take(limit) + /** + * A custom implementation modeled after the take function on RDDs but which never runs any job + * locally. This is to avoid shipping an entire partition of data in order to retrieve only a few + * rows. + */ + override def executeCollect(): Array[Row] = { + if (limit == 0) { + return new Array[Row](0) + } + + val childRDD = child.execute().map(_.copy()) + + val buf = new ArrayBuffer[Row] + val totalParts = childRDD.partitions.length + var partsScanned = 0 + while (buf.size < limit && partsScanned < totalParts) { + // The number of partitions to try in this iteration. It is ok for this number to be + // greater than totalParts because we actually cap it at totalParts in runJob. + var numPartsToTry = 1 + if (partsScanned > 0) { + // If we didn't find any rows after the first iteration, just try all partitions next. + // Otherwise, interpolate the number of partitions we need to try, but overestimate it + // by 50%. + if (buf.size == 0) { + numPartsToTry = totalParts - 1 + } else { + numPartsToTry = (1.5 * limit * partsScanned / buf.size).toInt + } + } + numPartsToTry = math.max(0, numPartsToTry) // guard against negative num of partitions + + val left = limit - buf.size + val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts) + val sc = sqlContext.sparkContext + val res = + sc.runJob(childRDD, (it: Iterator[Row]) => it.take(left).toArray, p, allowLocal = false) + + res.foreach(buf ++= _.take(limit - buf.size)) + partsScanned += numPartsToTry + } + + buf.toArray + } override def execute() = { val rdd = child.execute().mapPartitions { iter => From 56e009d4f05d990c60e109838fa70457f97f44aa Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Jul 2014 16:49:31 +0900 Subject: [PATCH 115/124] [EC2] Add default history server port to ec2 script Right now I have to open it manually Author: Andrew Or Closes #1296 from andrewor14/hist-serv-port and squashes the following commits: 8895a1f [Andrew Or] Add default history server port to ec2 script --- ec2/spark_ec2.py | 1 + 1 file changed, 1 insertion(+) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index e22d93bd31bc2..f5c2bfb697c81 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -279,6 +279,7 @@ def launch_cluster(conn, opts, cluster_name): master_group.authorize(src_group=slave_group) master_group.authorize('tcp', 22, 22, '0.0.0.0/0') master_group.authorize('tcp', 8080, 8081, '0.0.0.0/0') + master_group.authorize('tcp', 18080, 18080, '0.0.0.0/0') master_group.authorize('tcp', 19999, 19999, '0.0.0.0/0') master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0') master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0') From cc3e0a14daf756ff5c2d4e7916438e175046e5bb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 8 Jul 2014 10:36:18 -0700 Subject: [PATCH 116/124] [SPARK-2395][SQL] Optimize common LIKE patterns. Author: Michael Armbrust Closes #1325 from marmbrus/slowLike and squashes the following commits: 023c3eb [Michael Armbrust] add comment. 8b421c2 [Michael Armbrust] Handle the case where the final % is actually escaped. d34d37e [Michael Armbrust] add periods. 3bbf35f [Michael Armbrust] Roll back changes to SparkBuild 53894b1 [Michael Armbrust] Fix grammar. 4094462 [Michael Armbrust] Fix grammar. 6d3d0a0 [Michael Armbrust] Optimize common LIKE patterns. --- .../expressions/stringOperations.scala | 51 +++++++++++++++++++ .../sql/catalyst/optimizer/Optimizer.scala | 23 +++++++++ 2 files changed, 74 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index c074b7bb01e57..347471cebdc7e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -156,3 +156,54 @@ case class Lower(child: Expression) extends UnaryExpression with CaseConversionE override def toString() = s"Lower($child)" } + +/** A base class for functions that compare two strings, returning a boolean. */ +abstract class StringComparison extends Expression { + self: Product => + + type EvaluatedType = Any + + def left: Expression + def right: Expression + + override def references = children.flatMap(_.references).toSet + override def children = left :: right :: Nil + + override def nullable: Boolean = true + override def dataType: DataType = BooleanType + + def compare(l: String, r: String): Boolean + + override def eval(input: Row): Any = { + val leftEval = left.eval(input).asInstanceOf[String] + if(leftEval == null) { + null + } else { + val rightEval = right.eval(input).asInstanceOf[String] + if (rightEval == null) null else compare(leftEval, rightEval) + } + } + + override def toString() = s"$nodeName($left, $right)" +} + +/** + * A function that returns true if the string `left` contains the string `right`. + */ +case class Contains(left: Expression, right: Expression) extends StringComparison { + override def compare(l: String, r: String) = l.contains(r) +} + +/** + * A function that returns true if the string `left` starts with the string `right`. + */ +case class StartsWith(left: Expression, right: Expression) extends StringComparison { + def compare(l: String, r: String) = l.startsWith(r) +} + +/** + * A function that returns true if the string `left` ends with the string `right`. + */ +case class EndsWith(left: Expression, right: Expression) extends StringComparison { + def compare(l: String, r: String) = l.endsWith(r) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 48ca31e70fe7c..f0904f59d028f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -34,6 +34,7 @@ object Optimizer extends RuleExecutor[LogicalPlan] { Batch("ConstantFolding", FixedPoint(100), NullPropagation, ConstantFolding, + LikeSimplification, BooleanSimplification, SimplifyFilters, SimplifyCasts, @@ -111,6 +112,28 @@ object ColumnPruning extends Rule[LogicalPlan] { } } +/** + * Simplifies LIKE expressions that do not need full regular expressions to evaluate the condition. + * For example, when the expression is just checking to see if a string starts with a given + * pattern. + */ +object LikeSimplification extends Rule[LogicalPlan] { + // if guards below protect from escapes on trailing %. + // Cases like "something\%" are not optimized, but this does not affect correctness. + val startsWith = "([^_%]+)%".r + val endsWith = "%([^_%]+)".r + val contains = "%([^_%]+)%".r + + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case Like(l, Literal(startsWith(pattern), StringType)) if !pattern.endsWith("\\") => + StartsWith(l, Literal(pattern)) + case Like(l, Literal(endsWith(pattern), StringType)) => + EndsWith(l, Literal(pattern)) + case Like(l, Literal(contains(pattern), StringType)) if !pattern.endsWith("\\") => + Contains(l, Literal(pattern)) + } +} + /** * Replaces [[Expression Expressions]] that can be statically evaluated with * equivalent [[Literal]] values. This rule is more specific with From c8a2313cdf825e0191680a423d17619b5504ff89 Mon Sep 17 00:00:00 2001 From: Daniel Darabos Date: Tue, 8 Jul 2014 10:43:46 -0700 Subject: [PATCH 117/124] [SPARK-2403] Catch all errors during serialization in DAGScheduler https://issues.apache.org/jira/browse/SPARK-2403 Spark hangs for us whenever we forget to register a class with Kryo. This should be a simple fix for that. But let me know if you have a better suggestion. I did not write a new test for this. It would be pretty complicated and I'm not sure it's worthwhile for such a simple change. Let me know if you disagree. Author: Daniel Darabos Closes #1329 from darabos/spark-2403 and squashes the following commits: 3aceaad [Daniel Darabos] Print full stack trace for miscellaneous exceptions during serialization. 52c22ba [Daniel Darabos] Only catch NonFatal exceptions. 361e962 [Daniel Darabos] Catch all errors during serialization in DAGScheduler. --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 5 +++++ 1 file changed, 5 insertions(+) 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 81c136d970312..f72bfde572c96 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -26,6 +26,7 @@ import scala.concurrent.Await import scala.concurrent.duration._ import scala.language.postfixOps import scala.reflect.ClassTag +import scala.util.control.NonFatal import akka.actor._ import akka.actor.OneForOneStrategy @@ -768,6 +769,10 @@ class DAGScheduler( abortStage(stage, "Task not serializable: " + e.toString) runningStages -= stage return + case NonFatal(e) => // Other exceptions, such as IllegalArgumentException from Kryo. + abortStage(stage, s"Task serialization failed: $e\n${e.getStackTraceString}") + runningStages -= stage + return } logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") From b520b6453ed76926108e0bdd56114d16e1d86850 Mon Sep 17 00:00:00 2001 From: CrazyJvm Date: Tue, 8 Jul 2014 13:55:42 -0500 Subject: [PATCH 118/124] SPARK-2400 : fix spark.yarn.max.executor.failures explaination According to ```scala private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) ``` default value should be numExecutors * 2, with minimum of 3, and it's same to the config `spark.yarn.max.worker.failures` Author: CrazyJvm Closes #1282 from CrazyJvm/yarn-doc and squashes the following commits: 1a5f25b [CrazyJvm] remove deprecated config c438aec [CrazyJvm] fix style 86effa6 [CrazyJvm] change expression 211f130 [CrazyJvm] fix html tag 2900d23 [CrazyJvm] fix style a4b2e27 [CrazyJvm] fix configuration spark.yarn.max.executor.failures --- docs/running-on-yarn.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 5d8d603aa3e37..0362f5a223319 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -55,7 +55,7 @@ Most of the configs are the same for Spark on YARN as for other deployment modes - + From 32516f866a32d51bfaa04685ae77ba216b4202d9 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 8 Jul 2014 14:00:47 -0700 Subject: [PATCH 119/124] [SPARK-2409] Make SQLConf thread safe. Author: Reynold Xin Closes #1334 from rxin/sqlConfThreadSafetuy and squashes the following commits: c1e0a5a [Reynold Xin] Fixed the duplicate comment. 7614372 [Reynold Xin] [SPARK-2409] Make SQLConf thread safe. --- .../src/main/scala/org/apache/spark/sql/SQLConf.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) 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 95ed0f28507fc..b6fb46a3acc03 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 @@ -25,7 +25,9 @@ import scala.collection.JavaConverters._ * SQLConf holds mutable config parameters and hints. These can be set and * queried either by passing SET commands into Spark SQL's DSL * functions (sql(), hql(), etc.), or by programmatically using setters and - * getters of this class. This class is thread-safe. + * getters of this class. + * + * SQLConf is thread-safe (internally synchronized so safe to be used in multiple threads). */ trait SQLConf { @@ -71,11 +73,9 @@ trait SQLConf { Option(settings.get(key)).getOrElse(defaultValue) } - def getAll: Array[(String, String)] = settings.asScala.toArray + def getAll: Array[(String, String)] = settings.synchronized { settings.asScala.toArray } - def getOption(key: String): Option[String] = { - Option(settings.get(key)) - } + def getOption(key: String): Option[String] = Option(settings.get(key)) def contains(key: String): Boolean = settings.containsKey(key) From e6f7bfcfbf6aff7a9f8cd8e0a2166d0bf62b0912 Mon Sep 17 00:00:00 2001 From: Gabriele Nizzoli Date: Tue, 8 Jul 2014 14:23:38 -0700 Subject: [PATCH 120/124] [SPARK-2362] Fix for newFilesOnly logic in file DStream The newFilesOnly logic should be inverted: the logic should be that if the flag newFilesOnly==true then only start reading files older than current time. As the code is now if newFilesOnly==true then it will start to read files that are older than 0L (that is: every file in the directory). Author: Gabriele Nizzoli Closes #1077 from gabrielenizzoli/master and squashes the following commits: 4f1d261 [Gabriele Nizzoli] Fix for newFilesOnly logic in file DStream --- .../org/apache/spark/streaming/dstream/FileInputDStream.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index e878285f6a854..9eecbfaef363f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -45,7 +45,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas // Files with mod time earlier than this is ignored. This is updated every interval // such that in the current interval, files older than any file found in the // previous interval will be ignored. Obviously this time keeps moving forward. - private var ignoreTime = if (newFilesOnly) 0L else System.currentTimeMillis() + private var ignoreTime = if (newFilesOnly) System.currentTimeMillis() else 0L // Latest file mod time seen till any point of time @transient private var path_ : Path = null From bf04a390e40d60aa7fcc551501d25f3f9d38377c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Jul 2014 17:35:31 -0700 Subject: [PATCH 121/124] [SPARK-2392] Executors should not start their own HTTP servers Executors currently start their own unused HTTP file servers. This is because we use the same SparkEnv class for both executors and drivers, and we do not distinguish this case. In the longer term, we should separate out SparkEnv for the driver and SparkEnv for the executors. Author: Andrew Or Closes #1335 from andrewor14/executor-http-server and squashes the following commits: 46ef263 [Andrew Or] Start HTTP server only on the driver --- .../src/main/scala/org/apache/spark/SparkEnv.scala | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 2b636b085d73a..8f70744d804d9 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -79,7 +79,7 @@ class SparkEnv ( private[spark] def stop() { pythonWorkers.foreach { case(key, worker) => worker.stop() } - httpFileServer.stop() + Option(httpFileServer).foreach(_.stop()) mapOutputTracker.stop() shuffleManager.stop() broadcastManager.stop() @@ -228,9 +228,15 @@ object SparkEnv extends Logging { val cacheManager = new CacheManager(blockManager) - val httpFileServer = new HttpFileServer(securityManager) - httpFileServer.initialize() - conf.set("spark.fileserver.uri", httpFileServer.serverUri) + val httpFileServer = + if (isDriver) { + val server = new HttpFileServer(securityManager) + server.initialize() + conf.set("spark.fileserver.uri", server.serverUri) + server + } else { + null + } val metricsSystem = if (isDriver) { MetricsSystem.createMetricsSystem("driver", conf, securityManager) From ac9cdc116e1c5fcb291a4ff168cac002a8058f05 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 8 Jul 2014 17:50:36 -0700 Subject: [PATCH 122/124] [SPARK-2413] Upgrade junit_xml_listener to 0.5.1 which fixes the following issues 1) fix the class name to be fully qualified classpath 2) make sure the the reporting time is in second not in miliseond, which causing JUnit HTML to report incorrect number 3) make sure the duration of the tests are accumulative. Author: DB Tsai Closes #1333 from dbtsai/dbtsai-junit and squashes the following commits: bbeac4b [DB Tsai] Upgrade junit_xml_listener to 0.5.1 which fixes the following issues --- project/plugins.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/plugins.sbt b/project/plugins.sbt index 472819b9fb8ba..d3ac4bf335e87 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -21,6 +21,6 @@ addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.4.0") addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") -addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.0") +addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.1") addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.0") From 1114207cc8e4ef94cb97bbd5a2ef3ae4d51f73fa Mon Sep 17 00:00:00 2001 From: johnnywalleye Date: Tue, 8 Jul 2014 19:17:26 -0700 Subject: [PATCH 123/124] [SPARK-2152][MLlib] fix bin offset in DecisionTree node aggregations (also resolves SPARK-2160) Hi, this pull fixes (what I believe to be) a bug in DecisionTree.scala. In the extractLeftRightNodeAggregates function, the first set of rightNodeAgg values for Regression are set in line 792 as follows: rightNodeAgg(featureIndex)(2 * (numBins - 2)) = binData(shift + (2 * numBins - 1))) Then there is a loop that sets the rest of the values, as in line 809: rightNodeAgg(featureIndex)(2 * (numBins - 2 - splitIndex)) = binData(shift + (2 *(numBins - 2 - splitIndex))) + rightNodeAgg(featureIndex)(2 * (numBins - 1 - splitIndex)) But since splitIndex starts at 1, this ends up skipping a set of binData values. The changes here address this issue, for both the Regression and Classification cases. Author: johnnywalleye Closes #1316 from johnnywalleye/master and squashes the following commits: 73809da [johnnywalleye] fix bin offset in DecisionTree node aggregations --- .../org/apache/spark/mllib/tree/DecisionTree.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 3b13e52a7b445..74d5d7ba10960 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -807,10 +807,10 @@ object DecisionTree extends Serializable with Logging { // calculating right node aggregate for a split as a sum of right node aggregate of a // higher split and the right bin aggregate of a bin where the split is a low split rightNodeAgg(featureIndex)(2 * (numBins - 2 - splitIndex)) = - binData(shift + (2 *(numBins - 2 - splitIndex))) + + binData(shift + (2 *(numBins - 1 - splitIndex))) + rightNodeAgg(featureIndex)(2 * (numBins - 1 - splitIndex)) rightNodeAgg(featureIndex)(2 * (numBins - 2 - splitIndex) + 1) = - binData(shift + (2* (numBins - 2 - splitIndex) + 1)) + + binData(shift + (2* (numBins - 1 - splitIndex) + 1)) + rightNodeAgg(featureIndex)(2 * (numBins - 1 - splitIndex) + 1) splitIndex += 1 @@ -855,13 +855,13 @@ object DecisionTree extends Serializable with Logging { // calculating right node aggregate for a split as a sum of right node aggregate of a // higher split and the right bin aggregate of a bin where the split is a low split rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex)) = - binData(shift + (3 * (numBins - 2 - splitIndex))) + + binData(shift + (3 * (numBins - 1 - splitIndex))) + rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex)) rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex) + 1) = - binData(shift + (3 * (numBins - 2 - splitIndex) + 1)) + + binData(shift + (3 * (numBins - 1 - splitIndex) + 1)) + rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex) + 1) rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex) + 2) = - binData(shift + (3 * (numBins - 2 - splitIndex) + 2)) + + binData(shift + (3 * (numBins - 1 - splitIndex) + 2)) + rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex) + 2) splitIndex += 1 From 339441f545ca96de4a7d6271e32f16803539a8c8 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 8 Jul 2014 22:57:21 -0700 Subject: [PATCH 124/124] [SPARK-2384] Add tooltips to UI. This patch adds tooltips to clarify some points of confusion in the UI. When users mouse over some of the table headers (shuffle read, write, and input size) as well as over the "scheduler delay" metric shown for each stage, a black tool tip (see image below) pops up describing the metric in more detail. After the tooltip mechanism is added by this commit, I imagine others may want to add more tooltips for other things in the UI, but I think this is a good starting point. ![tooltip](https://cloud.githubusercontent.com/assets/1108612/3491905/994e179e-059f-11e4-92f2-c6c12d248d81.jpg) This looks scary-big but much of it is adding the bootstrap tool tip JavaScript. Also I have no idea what to put for the license in tooltip (I left it the same -- the Twitter apache header) or for JQuery (left it as nothing) -- @mateiz what's the right thing here? cc @pwendell @andrewor14 @rxin Author: Kay Ousterhout Closes #1314 from kayousterhout/tooltips and squashes the following commits: 19981b5 [Kay Ousterhout] Exclude non-licensed javascript files from style check d9ab5a9 [Kay Ousterhout] Response to Andrew's review 7752449 [Kay Ousterhout] [SPARK-2384] Add tooltips to UI. --- .rat-excludes | 2 + .../spark/ui/static/bootstrap-tooltip.js | 288 ++++++++++++++++++ .../spark/ui/static/initialize-tooltips.js | 21 ++ .../spark/ui/static/jquery-1.11.1.min.js | 4 + .../org/apache/spark/ui/static/webui.css | 5 + .../scala/org/apache/spark/ui/ToolTips.scala | 34 +++ .../scala/org/apache/spark/ui/UIUtils.scala | 32 +- .../apache/spark/ui/exec/ExecutorsPage.scala | 136 +++++---- .../apache/spark/ui/jobs/ExecutorTable.scala | 10 +- .../org/apache/spark/ui/jobs/StagePage.scala | 37 +-- .../org/apache/spark/ui/jobs/StageTable.scala | 15 +- 11 files changed, 491 insertions(+), 93 deletions(-) create mode 100644 core/src/main/resources/org/apache/spark/ui/static/bootstrap-tooltip.js create mode 100644 core/src/main/resources/org/apache/spark/ui/static/initialize-tooltips.js create mode 100644 core/src/main/resources/org/apache/spark/ui/static/jquery-1.11.1.min.js create mode 100644 core/src/main/scala/org/apache/spark/ui/ToolTips.scala diff --git a/.rat-excludes b/.rat-excludes index 796c32a80896c..372bc2587ccc3 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -22,6 +22,8 @@ slaves spark-env.sh spark-env.sh.template log4j-defaults.properties +bootstrap-tooltip.js +jquery-1.11.1.min.js sorttable.js .*txt .*json diff --git a/core/src/main/resources/org/apache/spark/ui/static/bootstrap-tooltip.js b/core/src/main/resources/org/apache/spark/ui/static/bootstrap-tooltip.js new file mode 100644 index 0000000000000..2934181c1006a --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/bootstrap-tooltip.js @@ -0,0 +1,288 @@ +/* =========================================================== + * bootstrap-tooltip.js v2.2.2 + * http://twitter.github.com/bootstrap/javascript.html#tooltips + * Inspired by the original jQuery.tipsy by Jason Frame + * =========================================================== + * Copyright 2012 Twitter, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * ========================================================== */ + + +!function ($) { + + "use strict"; // jshint ;_; + + + /* TOOLTIP PUBLIC CLASS DEFINITION + * =============================== */ + + var Tooltip = function (element, options) { + this.init('tooltip', element, options) + } + + Tooltip.prototype = { + + constructor: Tooltip + + , init: function (type, element, options) { + var eventIn + , eventOut + + this.type = type + this.$element = $(element) + this.options = this.getOptions(options) + this.enabled = true + + if (this.options.trigger == 'click') { + this.$element.on('click.' + this.type, this.options.selector, $.proxy(this.toggle, this)) + } else if (this.options.trigger != 'manual') { + eventIn = this.options.trigger == 'hover' ? 'mouseenter' : 'focus' + eventOut = this.options.trigger == 'hover' ? 'mouseleave' : 'blur' + this.$element.on(eventIn + '.' + this.type, this.options.selector, $.proxy(this.enter, this)) + this.$element.on(eventOut + '.' + this.type, this.options.selector, $.proxy(this.leave, this)) + } + + this.options.selector ? + (this._options = $.extend({}, this.options, { trigger: 'manual', selector: '' })) : + this.fixTitle() + } + + , getOptions: function (options) { + options = $.extend({}, $.fn[this.type].defaults, options, this.$element.data()) + + if (options.delay && typeof options.delay == 'number') { + options.delay = { + show: options.delay + , hide: options.delay + } + } + + return options + } + + , enter: function (e) { + var self = $(e.currentTarget)[this.type](this._options).data(this.type) + + if (!self.options.delay || !self.options.delay.show) return self.show() + + clearTimeout(this.timeout) + self.hoverState = 'in' + this.timeout = setTimeout(function() { + if (self.hoverState == 'in') self.show() + }, self.options.delay.show) + } + + , leave: function (e) { + var self = $(e.currentTarget)[this.type](this._options).data(this.type) + + if (this.timeout) clearTimeout(this.timeout) + if (!self.options.delay || !self.options.delay.hide) return self.hide() + + self.hoverState = 'out' + this.timeout = setTimeout(function() { + if (self.hoverState == 'out') self.hide() + }, self.options.delay.hide) + } + + , show: function () { + var $tip + , inside + , pos + , actualWidth + , actualHeight + , placement + , tp + + if (this.hasContent() && this.enabled) { + $tip = this.tip() + this.setContent() + + if (this.options.animation) { + $tip.addClass('fade') + } + + placement = typeof this.options.placement == 'function' ? + this.options.placement.call(this, $tip[0], this.$element[0]) : + this.options.placement + + inside = /in/.test(placement) + + $tip + .detach() + .css({ top: 0, left: 0, display: 'block' }) + .insertAfter(this.$element) + + pos = this.getPosition(inside) + + actualWidth = $tip[0].offsetWidth + actualHeight = $tip[0].offsetHeight + + switch (inside ? placement.split(' ')[1] : placement) { + case 'bottom': + tp = {top: pos.top + pos.height, left: pos.left + pos.width / 2 - actualWidth / 2} + break + case 'top': + tp = {top: pos.top - actualHeight, left: pos.left + pos.width / 2 - actualWidth / 2} + break + case 'left': + tp = {top: pos.top + pos.height / 2 - actualHeight / 2, left: pos.left - actualWidth} + break + case 'right': + tp = {top: pos.top + pos.height / 2 - actualHeight / 2, left: pos.left + pos.width} + break + } + + $tip + .offset(tp) + .addClass(placement) + .addClass('in') + } + } + + , setContent: function () { + var $tip = this.tip() + , title = this.getTitle() + + $tip.find('.tooltip-inner')[this.options.html ? 'html' : 'text'](title) + $tip.removeClass('fade in top bottom left right') + } + + , hide: function () { + var that = this + , $tip = this.tip() + + $tip.removeClass('in') + + function removeWithAnimation() { + var timeout = setTimeout(function () { + $tip.off($.support.transition.end).detach() + }, 500) + + $tip.one($.support.transition.end, function () { + clearTimeout(timeout) + $tip.detach() + }) + } + + $.support.transition && this.$tip.hasClass('fade') ? + removeWithAnimation() : + $tip.detach() + + return this + } + + , fixTitle: function () { + var $e = this.$element + if ($e.attr('title') || typeof($e.attr('data-original-title')) != 'string') { + $e.attr('data-original-title', $e.attr('title') || '').attr('title', '') + } + } + + , hasContent: function () { + return this.getTitle() + } + + , getPosition: function (inside) { + return $.extend({}, (inside ? {top: 0, left: 0} : this.$element.offset()), { + width: this.$element[0].offsetWidth + , height: this.$element[0].offsetHeight + }) + } + + , getTitle: function () { + var title + , $e = this.$element + , o = this.options + + title = $e.attr('data-original-title') + || (typeof o.title == 'function' ? o.title.call($e[0]) : o.title) + + return title + } + + , tip: function () { + return this.$tip = this.$tip || $(this.options.template) + } + + , validate: function () { + if (!this.$element[0].parentNode) { + this.hide() + this.$element = null + this.options = null + } + } + + , enable: function () { + this.enabled = true + } + + , disable: function () { + this.enabled = false + } + + , toggleEnabled: function () { + this.enabled = !this.enabled + } + + , toggle: function (e) { + var self = $(e.currentTarget)[this.type](this._options).data(this.type) + self[self.tip().hasClass('in') ? 'hide' : 'show']() + } + + , destroy: function () { + this.hide().$element.off('.' + this.type).removeData(this.type) + } + + } + + + /* TOOLTIP PLUGIN DEFINITION + * ========================= */ + + var old = $.fn.tooltip + + $.fn.tooltip = function ( option ) { + return this.each(function () { + var $this = $(this) + , data = $this.data('tooltip') + , options = typeof option == 'object' && option + if (!data) $this.data('tooltip', (data = new Tooltip(this, options))) + if (typeof option == 'string') data[option]() + }) + } + + $.fn.tooltip.Constructor = Tooltip + + $.fn.tooltip.defaults = { + animation: true + , placement: 'top' + , selector: false + , template: '
    ' + , trigger: 'hover' + , title: '' + , delay: 0 + , html: false + } + + + /* TOOLTIP NO CONFLICT + * =================== */ + + $.fn.tooltip.noConflict = function () { + $.fn.tooltip = old + return this + } + +}(window.jQuery); + diff --git a/core/src/main/resources/org/apache/spark/ui/static/initialize-tooltips.js b/core/src/main/resources/org/apache/spark/ui/static/initialize-tooltips.js new file mode 100644 index 0000000000000..70f355dfb49cb --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/initialize-tooltips.js @@ -0,0 +1,21 @@ +/* + * 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. + */ + +$(document).ready(function(){ + $("[data-toggle=tooltip]").tooltip({container: 'body'}); +}); + diff --git a/core/src/main/resources/org/apache/spark/ui/static/jquery-1.11.1.min.js b/core/src/main/resources/org/apache/spark/ui/static/jquery-1.11.1.min.js new file mode 100644 index 0000000000000..ab28a24729b32 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/jquery-1.11.1.min.js @@ -0,0 +1,4 @@ +/*! jQuery v1.11.1 | (c) 2005, 2014 jQuery Foundation, Inc. | jquery.org/license */ +!function(a,b){"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){var c=[],d=c.slice,e=c.concat,f=c.push,g=c.indexOf,h={},i=h.toString,j=h.hasOwnProperty,k={},l="1.11.1",m=function(a,b){return new m.fn.init(a,b)},n=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,o=/^-ms-/,p=/-([\da-z])/gi,q=function(a,b){return b.toUpperCase()};m.fn=m.prototype={jquery:l,constructor:m,selector:"",length:0,toArray:function(){return d.call(this)},get:function(a){return null!=a?0>a?this[a+this.length]:this[a]:d.call(this)},pushStack:function(a){var b=m.merge(this.constructor(),a);return b.prevObject=this,b.context=this.context,b},each:function(a,b){return m.each(this,a,b)},map:function(a){return this.pushStack(m.map(this,function(b,c){return a.call(b,c,b)}))},slice:function(){return this.pushStack(d.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(a){var b=this.length,c=+a+(0>a?b:0);return this.pushStack(c>=0&&b>c?[this[c]]:[])},end:function(){return this.prevObject||this.constructor(null)},push:f,sort:c.sort,splice:c.splice},m.extend=m.fn.extend=function(){var a,b,c,d,e,f,g=arguments[0]||{},h=1,i=arguments.length,j=!1;for("boolean"==typeof g&&(j=g,g=arguments[h]||{},h++),"object"==typeof g||m.isFunction(g)||(g={}),h===i&&(g=this,h--);i>h;h++)if(null!=(e=arguments[h]))for(d in e)a=g[d],c=e[d],g!==c&&(j&&c&&(m.isPlainObject(c)||(b=m.isArray(c)))?(b?(b=!1,f=a&&m.isArray(a)?a:[]):f=a&&m.isPlainObject(a)?a:{},g[d]=m.extend(j,f,c)):void 0!==c&&(g[d]=c));return g},m.extend({expando:"jQuery"+(l+Math.random()).replace(/\D/g,""),isReady:!0,error:function(a){throw new Error(a)},noop:function(){},isFunction:function(a){return"function"===m.type(a)},isArray:Array.isArray||function(a){return"array"===m.type(a)},isWindow:function(a){return null!=a&&a==a.window},isNumeric:function(a){return!m.isArray(a)&&a-parseFloat(a)>=0},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},isPlainObject:function(a){var b;if(!a||"object"!==m.type(a)||a.nodeType||m.isWindow(a))return!1;try{if(a.constructor&&!j.call(a,"constructor")&&!j.call(a.constructor.prototype,"isPrototypeOf"))return!1}catch(c){return!1}if(k.ownLast)for(b in a)return j.call(a,b);for(b in a);return void 0===b||j.call(a,b)},type:function(a){return null==a?a+"":"object"==typeof a||"function"==typeof a?h[i.call(a)]||"object":typeof a},globalEval:function(b){b&&m.trim(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(o,"ms-").replace(p,q)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()},each:function(a,b,c){var d,e=0,f=a.length,g=r(a);if(c){if(g){for(;f>e;e++)if(d=b.apply(a[e],c),d===!1)break}else for(e in a)if(d=b.apply(a[e],c),d===!1)break}else if(g){for(;f>e;e++)if(d=b.call(a[e],e,a[e]),d===!1)break}else for(e in a)if(d=b.call(a[e],e,a[e]),d===!1)break;return a},trim:function(a){return null==a?"":(a+"").replace(n,"")},makeArray:function(a,b){var c=b||[];return null!=a&&(r(Object(a))?m.merge(c,"string"==typeof a?[a]:a):f.call(c,a)),c},inArray:function(a,b,c){var d;if(b){if(g)return g.call(b,a,c);for(d=b.length,c=c?0>c?Math.max(0,d+c):c:0;d>c;c++)if(c in b&&b[c]===a)return c}return-1},merge:function(a,b){var c=+b.length,d=0,e=a.length;while(c>d)a[e++]=b[d++];if(c!==c)while(void 0!==b[d])a[e++]=b[d++];return a.length=e,a},grep:function(a,b,c){for(var d,e=[],f=0,g=a.length,h=!c;g>f;f++)d=!b(a[f],f),d!==h&&e.push(a[f]);return e},map:function(a,b,c){var d,f=0,g=a.length,h=r(a),i=[];if(h)for(;g>f;f++)d=b(a[f],f,c),null!=d&&i.push(d);else for(f in a)d=b(a[f],f,c),null!=d&&i.push(d);return e.apply([],i)},guid:1,proxy:function(a,b){var c,e,f;return"string"==typeof b&&(f=a[b],b=a,a=f),m.isFunction(a)?(c=d.call(arguments,2),e=function(){return a.apply(b||this,c.concat(d.call(arguments)))},e.guid=a.guid=a.guid||m.guid++,e):void 0},now:function(){return+new Date},support:k}),m.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(a,b){h["[object "+b+"]"]=b.toLowerCase()});function r(a){var b=a.length,c=m.type(a);return"function"===c||m.isWindow(a)?!1:1===a.nodeType&&b?!0:"array"===c||0===b||"number"==typeof b&&b>0&&b-1 in a}var s=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u="sizzle"+-new Date,v=a.document,w=0,x=0,y=gb(),z=gb(),A=gb(),B=function(a,b){return a===b&&(l=!0),0},C="undefined",D=1<<31,E={}.hasOwnProperty,F=[],G=F.pop,H=F.push,I=F.push,J=F.slice,K=F.indexOf||function(a){for(var b=0,c=this.length;c>b;b++)if(this[b]===a)return b;return-1},L="checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",M="[\\x20\\t\\r\\n\\f]",N="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",O=N.replace("w","w#"),P="\\["+M+"*("+N+")(?:"+M+"*([*^$|!~]?=)"+M+"*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|("+O+"))|)"+M+"*\\]",Q=":("+N+")(?:\\((('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|((?:\\\\.|[^\\\\()[\\]]|"+P+")*)|.*)\\)|)",R=new RegExp("^"+M+"+|((?:^|[^\\\\])(?:\\\\.)*)"+M+"+$","g"),S=new RegExp("^"+M+"*,"+M+"*"),T=new RegExp("^"+M+"*([>+~]|"+M+")"+M+"*"),U=new RegExp("="+M+"*([^\\]'\"]*?)"+M+"*\\]","g"),V=new RegExp(Q),W=new RegExp("^"+O+"$"),X={ID:new RegExp("^#("+N+")"),CLASS:new RegExp("^\\.("+N+")"),TAG:new RegExp("^("+N.replace("w","w*")+")"),ATTR:new RegExp("^"+P),PSEUDO:new RegExp("^"+Q),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+M+"*(even|odd|(([+-]|)(\\d*)n|)"+M+"*(?:([+-]|)"+M+"*(\\d+)|))"+M+"*\\)|)","i"),bool:new RegExp("^(?:"+L+")$","i"),needsContext:new RegExp("^"+M+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+M+"*((?:-\\d)?\\d*)"+M+"*\\)|)(?=[^-]|$)","i")},Y=/^(?:input|select|textarea|button)$/i,Z=/^h\d$/i,$=/^[^{]+\{\s*\[native \w/,_=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,ab=/[+~]/,bb=/'|\\/g,cb=new RegExp("\\\\([\\da-f]{1,6}"+M+"?|("+M+")|.)","ig"),db=function(a,b,c){var d="0x"+b-65536;return d!==d||c?b:0>d?String.fromCharCode(d+65536):String.fromCharCode(d>>10|55296,1023&d|56320)};try{I.apply(F=J.call(v.childNodes),v.childNodes),F[v.childNodes.length].nodeType}catch(eb){I={apply:F.length?function(a,b){H.apply(a,J.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function fb(a,b,d,e){var f,h,j,k,l,o,r,s,w,x;if((b?b.ownerDocument||b:v)!==n&&m(b),b=b||n,d=d||[],!a||"string"!=typeof a)return d;if(1!==(k=b.nodeType)&&9!==k)return[];if(p&&!e){if(f=_.exec(a))if(j=f[1]){if(9===k){if(h=b.getElementById(j),!h||!h.parentNode)return d;if(h.id===j)return d.push(h),d}else if(b.ownerDocument&&(h=b.ownerDocument.getElementById(j))&&t(b,h)&&h.id===j)return d.push(h),d}else{if(f[2])return I.apply(d,b.getElementsByTagName(a)),d;if((j=f[3])&&c.getElementsByClassName&&b.getElementsByClassName)return I.apply(d,b.getElementsByClassName(j)),d}if(c.qsa&&(!q||!q.test(a))){if(s=r=u,w=b,x=9===k&&a,1===k&&"object"!==b.nodeName.toLowerCase()){o=g(a),(r=b.getAttribute("id"))?s=r.replace(bb,"\\$&"):b.setAttribute("id",s),s="[id='"+s+"'] ",l=o.length;while(l--)o[l]=s+qb(o[l]);w=ab.test(a)&&ob(b.parentNode)||b,x=o.join(",")}if(x)try{return I.apply(d,w.querySelectorAll(x)),d}catch(y){}finally{r||b.removeAttribute("id")}}}return i(a.replace(R,"$1"),b,d,e)}function gb(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function hb(a){return a[u]=!0,a}function ib(a){var b=n.createElement("div");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function jb(a,b){var c=a.split("|"),e=a.length;while(e--)d.attrHandle[c[e]]=b}function kb(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&(~b.sourceIndex||D)-(~a.sourceIndex||D);if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function lb(a){return function(b){var c=b.nodeName.toLowerCase();return"input"===c&&b.type===a}}function mb(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}function nb(a){return hb(function(b){return b=+b,hb(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function ob(a){return a&&typeof a.getElementsByTagName!==C&&a}c=fb.support={},f=fb.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?"HTML"!==b.nodeName:!1},m=fb.setDocument=function(a){var b,e=a?a.ownerDocument||a:v,g=e.defaultView;return e!==n&&9===e.nodeType&&e.documentElement?(n=e,o=e.documentElement,p=!f(e),g&&g!==g.top&&(g.addEventListener?g.addEventListener("unload",function(){m()},!1):g.attachEvent&&g.attachEvent("onunload",function(){m()})),c.attributes=ib(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=ib(function(a){return a.appendChild(e.createComment("")),!a.getElementsByTagName("*").length}),c.getElementsByClassName=$.test(e.getElementsByClassName)&&ib(function(a){return a.innerHTML="
    ",a.firstChild.className="i",2===a.getElementsByClassName("i").length}),c.getById=ib(function(a){return o.appendChild(a).id=u,!e.getElementsByName||!e.getElementsByName(u).length}),c.getById?(d.find.ID=function(a,b){if(typeof b.getElementById!==C&&p){var c=b.getElementById(a);return c&&c.parentNode?[c]:[]}},d.filter.ID=function(a){var b=a.replace(cb,db);return function(a){return a.getAttribute("id")===b}}):(delete d.find.ID,d.filter.ID=function(a){var b=a.replace(cb,db);return function(a){var c=typeof a.getAttributeNode!==C&&a.getAttributeNode("id");return c&&c.value===b}}),d.find.TAG=c.getElementsByTagName?function(a,b){return typeof b.getElementsByTagName!==C?b.getElementsByTagName(a):void 0}:function(a,b){var c,d=[],e=0,f=b.getElementsByTagName(a);if("*"===a){while(c=f[e++])1===c.nodeType&&d.push(c);return d}return f},d.find.CLASS=c.getElementsByClassName&&function(a,b){return typeof b.getElementsByClassName!==C&&p?b.getElementsByClassName(a):void 0},r=[],q=[],(c.qsa=$.test(e.querySelectorAll))&&(ib(function(a){a.innerHTML="",a.querySelectorAll("[msallowclip^='']").length&&q.push("[*^$]="+M+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||q.push("\\["+M+"*(?:value|"+L+")"),a.querySelectorAll(":checked").length||q.push(":checked")}),ib(function(a){var b=e.createElement("input");b.setAttribute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&q.push("name"+M+"*[*^$|!~]?="),a.querySelectorAll(":enabled").length||q.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),q.push(",.*:")})),(c.matchesSelector=$.test(s=o.matches||o.webkitMatchesSelector||o.mozMatchesSelector||o.oMatchesSelector||o.msMatchesSelector))&&ib(function(a){c.disconnectedMatch=s.call(a,"div"),s.call(a,"[s!='']:x"),r.push("!=",Q)}),q=q.length&&new RegExp(q.join("|")),r=r.length&&new RegExp(r.join("|")),b=$.test(o.compareDocumentPosition),t=b||$.test(o.contains)?function(a,b){var c=9===a.nodeType?a.documentElement:a,d=b&&b.parentNode;return a===d||!(!d||1!==d.nodeType||!(c.contains?c.contains(d):a.compareDocumentPosition&&16&a.compareDocumentPosition(d)))}:function(a,b){if(b)while(b=b.parentNode)if(b===a)return!0;return!1},B=b?function(a,b){if(a===b)return l=!0,0;var d=!a.compareDocumentPosition-!b.compareDocumentPosition;return d?d:(d=(a.ownerDocument||a)===(b.ownerDocument||b)?a.compareDocumentPosition(b):1,1&d||!c.sortDetached&&b.compareDocumentPosition(a)===d?a===e||a.ownerDocument===v&&t(v,a)?-1:b===e||b.ownerDocument===v&&t(v,b)?1:k?K.call(k,a)-K.call(k,b):0:4&d?-1:1)}:function(a,b){if(a===b)return l=!0,0;var c,d=0,f=a.parentNode,g=b.parentNode,h=[a],i=[b];if(!f||!g)return a===e?-1:b===e?1:f?-1:g?1:k?K.call(k,a)-K.call(k,b):0;if(f===g)return kb(a,b);c=a;while(c=c.parentNode)h.unshift(c);c=b;while(c=c.parentNode)i.unshift(c);while(h[d]===i[d])d++;return d?kb(h[d],i[d]):h[d]===v?-1:i[d]===v?1:0},e):n},fb.matches=function(a,b){return fb(a,null,null,b)},fb.matchesSelector=function(a,b){if((a.ownerDocument||a)!==n&&m(a),b=b.replace(U,"='$1']"),!(!c.matchesSelector||!p||r&&r.test(b)||q&&q.test(b)))try{var d=s.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return fb(b,n,null,[a]).length>0},fb.contains=function(a,b){return(a.ownerDocument||a)!==n&&m(a),t(a,b)},fb.attr=function(a,b){(a.ownerDocument||a)!==n&&m(a);var e=d.attrHandle[b.toLowerCase()],f=e&&E.call(d.attrHandle,b.toLowerCase())?e(a,b,!p):void 0;return void 0!==f?f:c.attributes||!p?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},fb.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},fb.uniqueSort=function(a){var b,d=[],e=0,f=0;if(l=!c.detectDuplicates,k=!c.sortStable&&a.slice(0),a.sort(B),l){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return k=null,a},e=fb.getText=function(a){var b,c="",d=0,f=a.nodeType;if(f){if(1===f||9===f||11===f){if("string"==typeof a.textContent)return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=e(a)}else if(3===f||4===f)return a.nodeValue}else while(b=a[d++])c+=e(b);return c},d=fb.selectors={cacheLength:50,createPseudo:hb,match:X,attrHandle:{},find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(cb,db),a[3]=(a[3]||a[4]||a[5]||"").replace(cb,db),"~="===a[2]&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),"nth"===a[1].slice(0,3)?(a[3]||fb.error(a[0]),a[4]=+(a[4]?a[5]+(a[6]||1):2*("even"===a[3]||"odd"===a[3])),a[5]=+(a[7]+a[8]||"odd"===a[3])):a[3]&&fb.error(a[0]),a},PSEUDO:function(a){var b,c=!a[6]&&a[2];return X.CHILD.test(a[0])?null:(a[3]?a[2]=a[4]||a[5]||"":c&&V.test(c)&&(b=g(c,!0))&&(b=c.indexOf(")",c.length-b)-c.length)&&(a[0]=a[0].slice(0,b),a[2]=c.slice(0,b)),a.slice(0,3))}},filter:{TAG:function(a){var b=a.replace(cb,db).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=y[a+" "];return b||(b=new RegExp("(^|"+M+")"+a+"("+M+"|$)"))&&y(a,function(a){return b.test("string"==typeof a.className&&a.className||typeof a.getAttribute!==C&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=fb.attr(d,a);return null==e?"!="===b:b?(e+="","="===b?e===c:"!="===b?e!==c:"^="===b?c&&0===e.indexOf(c):"*="===b?c&&e.indexOf(c)>-1:"$="===b?c&&e.slice(-c.length)===c:"~="===b?(" "+e+" ").indexOf(c)>-1:"|="===b?e===c||e.slice(0,c.length+1)===c+"-":!1):!0}},CHILD:function(a,b,c,d,e){var f="nth"!==a.slice(0,3),g="last"!==a.slice(-4),h="of-type"===b;return 1===d&&0===e?function(a){return!!a.parentNode}:function(b,c,i){var j,k,l,m,n,o,p=f!==g?"nextSibling":"previousSibling",q=b.parentNode,r=h&&b.nodeName.toLowerCase(),s=!i&&!h;if(q){if(f){while(p){l=b;while(l=l[p])if(h?l.nodeName.toLowerCase()===r:1===l.nodeType)return!1;o=p="only"===a&&!o&&"nextSibling"}return!0}if(o=[g?q.firstChild:q.lastChild],g&&s){k=q[u]||(q[u]={}),j=k[a]||[],n=j[0]===w&&j[1],m=j[0]===w&&j[2],l=n&&q.childNodes[n];while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if(1===l.nodeType&&++m&&l===b){k[a]=[w,n,m];break}}else if(s&&(j=(b[u]||(b[u]={}))[a])&&j[0]===w)m=j[1];else while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if((h?l.nodeName.toLowerCase()===r:1===l.nodeType)&&++m&&(s&&((l[u]||(l[u]={}))[a]=[w,m]),l===b))break;return m-=e,m===d||m%d===0&&m/d>=0}}},PSEUDO:function(a,b){var c,e=d.pseudos[a]||d.setFilters[a.toLowerCase()]||fb.error("unsupported pseudo: "+a);return e[u]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?hb(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=K.call(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:hb(function(a){var b=[],c=[],d=h(a.replace(R,"$1"));return d[u]?hb(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)(f=g[h])&&(a[h]=!(b[h]=f))}):function(a,e,f){return b[0]=a,d(b,null,f,c),!c.pop()}}),has:hb(function(a){return function(b){return fb(a,b).length>0}}),contains:hb(function(a){return function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:hb(function(a){return W.test(a||"")||fb.error("unsupported lang: "+a),a=a.replace(cb,db).toLowerCase(),function(b){var c;do if(c=p?b.lang:b.getAttribute("xml:lang")||b.getAttribute("lang"))return c=c.toLowerCase(),c===a||0===c.indexOf(a+"-");while((b=b.parentNode)&&1===b.nodeType);return!1}}),target:function(b){var c=a.location&&a.location.hash;return c&&c.slice(1)===b.id},root:function(a){return a===o},focus:function(a){return a===n.activeElement&&(!n.hasFocus||n.hasFocus())&&!!(a.type||a.href||~a.tabIndex)},enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&!!a.checked||"option"===b&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},empty:function(a){for(a=a.firstChild;a;a=a.nextSibling)if(a.nodeType<6)return!1;return!0},parent:function(a){return!d.pseudos.empty(a)},header:function(a){return Z.test(a.nodeName)},input:function(a){return Y.test(a.nodeName)},button:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&"button"===a.type||"button"===b},text:function(a){var b;return"input"===a.nodeName.toLowerCase()&&"text"===a.type&&(null==(b=a.getAttribute("type"))||"text"===b.toLowerCase())},first:nb(function(){return[0]}),last:nb(function(a,b){return[b-1]}),eq:nb(function(a,b,c){return[0>c?c+b:c]}),even:nb(function(a,b){for(var c=0;b>c;c+=2)a.push(c);return a}),odd:nb(function(a,b){for(var c=1;b>c;c+=2)a.push(c);return a}),lt:nb(function(a,b,c){for(var d=0>c?c+b:c;--d>=0;)a.push(d);return a}),gt:nb(function(a,b,c){for(var d=0>c?c+b:c;++db;b++)d+=a[b].value;return d}function rb(a,b,c){var d=b.dir,e=c&&"parentNode"===d,f=x++;return b.first?function(b,c,f){while(b=b[d])if(1===b.nodeType||e)return a(b,c,f)}:function(b,c,g){var h,i,j=[w,f];if(g){while(b=b[d])if((1===b.nodeType||e)&&a(b,c,g))return!0}else while(b=b[d])if(1===b.nodeType||e){if(i=b[u]||(b[u]={}),(h=i[d])&&h[0]===w&&h[1]===f)return j[2]=h[2];if(i[d]=j,j[2]=a(b,c,g))return!0}}}function sb(a){return a.length>1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function tb(a,b,c){for(var d=0,e=b.length;e>d;d++)fb(a,b[d],c);return c}function ub(a,b,c,d,e){for(var f,g=[],h=0,i=a.length,j=null!=b;i>h;h++)(f=a[h])&&(!c||c(f,d,e))&&(g.push(f),j&&b.push(h));return g}function vb(a,b,c,d,e,f){return d&&!d[u]&&(d=vb(d)),e&&!e[u]&&(e=vb(e,f)),hb(function(f,g,h,i){var j,k,l,m=[],n=[],o=g.length,p=f||tb(b||"*",h.nodeType?[h]:h,[]),q=!a||!f&&b?p:ub(p,m,a,h,i),r=c?e||(f?a:o||d)?[]:g:q;if(c&&c(q,r,h,i),d){j=ub(r,n),d(j,[],h,i),k=j.length;while(k--)(l=j[k])&&(r[n[k]]=!(q[n[k]]=l))}if(f){if(e||a){if(e){j=[],k=r.length;while(k--)(l=r[k])&&j.push(q[k]=l);e(null,r=[],j,i)}k=r.length;while(k--)(l=r[k])&&(j=e?K.call(f,l):m[k])>-1&&(f[j]=!(g[j]=l))}}else r=ub(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):I.apply(g,r)})}function wb(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],h=g||d.relative[" "],i=g?1:0,k=rb(function(a){return a===b},h,!0),l=rb(function(a){return K.call(b,a)>-1},h,!0),m=[function(a,c,d){return!g&&(d||c!==j)||((b=c).nodeType?k(a,c,d):l(a,c,d))}];f>i;i++)if(c=d.relative[a[i].type])m=[rb(sb(m),c)];else{if(c=d.filter[a[i].type].apply(null,a[i].matches),c[u]){for(e=++i;f>e;e++)if(d.relative[a[e].type])break;return vb(i>1&&sb(m),i>1&&qb(a.slice(0,i-1).concat({value:" "===a[i-2].type?"*":""})).replace(R,"$1"),c,e>i&&wb(a.slice(i,e)),f>e&&wb(a=a.slice(e)),f>e&&qb(a))}m.push(c)}return sb(m)}function xb(a,b){var c=b.length>0,e=a.length>0,f=function(f,g,h,i,k){var l,m,o,p=0,q="0",r=f&&[],s=[],t=j,u=f||e&&d.find.TAG("*",k),v=w+=null==t?1:Math.random()||.1,x=u.length;for(k&&(j=g!==n&&g);q!==x&&null!=(l=u[q]);q++){if(e&&l){m=0;while(o=a[m++])if(o(l,g,h)){i.push(l);break}k&&(w=v)}c&&((l=!o&&l)&&p--,f&&r.push(l))}if(p+=q,c&&q!==p){m=0;while(o=b[m++])o(r,s,g,h);if(f){if(p>0)while(q--)r[q]||s[q]||(s[q]=G.call(i));s=ub(s)}I.apply(i,s),k&&!f&&s.length>0&&p+b.length>1&&fb.uniqueSort(i)}return k&&(w=v,j=t),r};return c?hb(f):f}return h=fb.compile=function(a,b){var c,d=[],e=[],f=A[a+" "];if(!f){b||(b=g(a)),c=b.length;while(c--)f=wb(b[c]),f[u]?d.push(f):e.push(f);f=A(a,xb(e,d)),f.selector=a}return f},i=fb.select=function(a,b,e,f){var i,j,k,l,m,n="function"==typeof a&&a,o=!f&&g(a=n.selector||a);if(e=e||[],1===o.length){if(j=o[0]=o[0].slice(0),j.length>2&&"ID"===(k=j[0]).type&&c.getById&&9===b.nodeType&&p&&d.relative[j[1].type]){if(b=(d.find.ID(k.matches[0].replace(cb,db),b)||[])[0],!b)return e;n&&(b=b.parentNode),a=a.slice(j.shift().value.length)}i=X.needsContext.test(a)?0:j.length;while(i--){if(k=j[i],d.relative[l=k.type])break;if((m=d.find[l])&&(f=m(k.matches[0].replace(cb,db),ab.test(j[0].type)&&ob(b.parentNode)||b))){if(j.splice(i,1),a=f.length&&qb(j),!a)return I.apply(e,f),e;break}}}return(n||h(a,o))(f,b,!p,e,ab.test(a)&&ob(b.parentNode)||b),e},c.sortStable=u.split("").sort(B).join("")===u,c.detectDuplicates=!!l,m(),c.sortDetached=ib(function(a){return 1&a.compareDocumentPosition(n.createElement("div"))}),ib(function(a){return a.innerHTML="","#"===a.firstChild.getAttribute("href")})||jb("type|href|height|width",function(a,b,c){return c?void 0:a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&ib(function(a){return a.innerHTML="",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||jb("value",function(a,b,c){return c||"input"!==a.nodeName.toLowerCase()?void 0:a.defaultValue}),ib(function(a){return null==a.getAttribute("disabled")})||jb(L,function(a,b,c){var d;return c?void 0:a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),fb}(a);m.find=s,m.expr=s.selectors,m.expr[":"]=m.expr.pseudos,m.unique=s.uniqueSort,m.text=s.getText,m.isXMLDoc=s.isXML,m.contains=s.contains;var t=m.expr.match.needsContext,u=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,v=/^.[^:#\[\.,]*$/;function w(a,b,c){if(m.isFunction(b))return m.grep(a,function(a,d){return!!b.call(a,d,a)!==c});if(b.nodeType)return m.grep(a,function(a){return a===b!==c});if("string"==typeof b){if(v.test(b))return m.filter(b,a,c);b=m.filter(b,a)}return m.grep(a,function(a){return m.inArray(a,b)>=0!==c})}m.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?m.find.matchesSelector(d,a)?[d]:[]:m.find.matches(a,m.grep(b,function(a){return 1===a.nodeType}))},m.fn.extend({find:function(a){var b,c=[],d=this,e=d.length;if("string"!=typeof a)return this.pushStack(m(a).filter(function(){for(b=0;e>b;b++)if(m.contains(d[b],this))return!0}));for(b=0;e>b;b++)m.find(a,d[b],c);return c=this.pushStack(e>1?m.unique(c):c),c.selector=this.selector?this.selector+" "+a:a,c},filter:function(a){return this.pushStack(w(this,a||[],!1))},not:function(a){return this.pushStack(w(this,a||[],!0))},is:function(a){return!!w(this,"string"==typeof a&&t.test(a)?m(a):a||[],!1).length}});var x,y=a.document,z=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,A=m.fn.init=function(a,b){var c,d;if(!a)return this;if("string"==typeof a){if(c="<"===a.charAt(0)&&">"===a.charAt(a.length-1)&&a.length>=3?[null,a,null]:z.exec(a),!c||!c[1]&&b)return!b||b.jquery?(b||x).find(a):this.constructor(b).find(a);if(c[1]){if(b=b instanceof m?b[0]:b,m.merge(this,m.parseHTML(c[1],b&&b.nodeType?b.ownerDocument||b:y,!0)),u.test(c[1])&&m.isPlainObject(b))for(c in b)m.isFunction(this[c])?this[c](b[c]):this.attr(c,b[c]);return this}if(d=y.getElementById(c[2]),d&&d.parentNode){if(d.id!==c[2])return x.find(a);this.length=1,this[0]=d}return this.context=y,this.selector=a,this}return a.nodeType?(this.context=this[0]=a,this.length=1,this):m.isFunction(a)?"undefined"!=typeof x.ready?x.ready(a):a(m):(void 0!==a.selector&&(this.selector=a.selector,this.context=a.context),m.makeArray(a,this))};A.prototype=m.fn,x=m(y);var B=/^(?:parents|prev(?:Until|All))/,C={children:!0,contents:!0,next:!0,prev:!0};m.extend({dir:function(a,b,c){var d=[],e=a[b];while(e&&9!==e.nodeType&&(void 0===c||1!==e.nodeType||!m(e).is(c)))1===e.nodeType&&d.push(e),e=e[b];return d},sibling:function(a,b){for(var c=[];a;a=a.nextSibling)1===a.nodeType&&a!==b&&c.push(a);return c}}),m.fn.extend({has:function(a){var b,c=m(a,this),d=c.length;return this.filter(function(){for(b=0;d>b;b++)if(m.contains(this,c[b]))return!0})},closest:function(a,b){for(var c,d=0,e=this.length,f=[],g=t.test(a)||"string"!=typeof a?m(a,b||this.context):0;e>d;d++)for(c=this[d];c&&c!==b;c=c.parentNode)if(c.nodeType<11&&(g?g.index(c)>-1:1===c.nodeType&&m.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?m.unique(f):f)},index:function(a){return a?"string"==typeof a?m.inArray(this[0],m(a)):m.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(m.unique(m.merge(this.get(),m(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function D(a,b){do a=a[b];while(a&&1!==a.nodeType);return a}m.each({parent:function(a){var b=a.parentNode;return b&&11!==b.nodeType?b:null},parents:function(a){return m.dir(a,"parentNode")},parentsUntil:function(a,b,c){return m.dir(a,"parentNode",c)},next:function(a){return D(a,"nextSibling")},prev:function(a){return D(a,"previousSibling")},nextAll:function(a){return m.dir(a,"nextSibling")},prevAll:function(a){return m.dir(a,"previousSibling")},nextUntil:function(a,b,c){return m.dir(a,"nextSibling",c)},prevUntil:function(a,b,c){return m.dir(a,"previousSibling",c)},siblings:function(a){return m.sibling((a.parentNode||{}).firstChild,a)},children:function(a){return m.sibling(a.firstChild)},contents:function(a){return m.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:m.merge([],a.childNodes)}},function(a,b){m.fn[a]=function(c,d){var e=m.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=m.filter(d,e)),this.length>1&&(C[a]||(e=m.unique(e)),B.test(a)&&(e=e.reverse())),this.pushStack(e)}});var E=/\S+/g,F={};function G(a){var b=F[a]={};return m.each(a.match(E)||[],function(a,c){b[c]=!0}),b}m.Callbacks=function(a){a="string"==typeof a?F[a]||G(a):m.extend({},a);var b,c,d,e,f,g,h=[],i=!a.once&&[],j=function(l){for(c=a.memory&&l,d=!0,f=g||0,g=0,e=h.length,b=!0;h&&e>f;f++)if(h[f].apply(l[0],l[1])===!1&&a.stopOnFalse){c=!1;break}b=!1,h&&(i?i.length&&j(i.shift()):c?h=[]:k.disable())},k={add:function(){if(h){var d=h.length;!function f(b){m.each(b,function(b,c){var d=m.type(c);"function"===d?a.unique&&k.has(c)||h.push(c):c&&c.length&&"string"!==d&&f(c)})}(arguments),b?e=h.length:c&&(g=d,j(c))}return this},remove:function(){return h&&m.each(arguments,function(a,c){var d;while((d=m.inArray(c,h,d))>-1)h.splice(d,1),b&&(e>=d&&e--,f>=d&&f--)}),this},has:function(a){return a?m.inArray(a,h)>-1:!(!h||!h.length)},empty:function(){return h=[],e=0,this},disable:function(){return h=i=c=void 0,this},disabled:function(){return!h},lock:function(){return i=void 0,c||k.disable(),this},locked:function(){return!i},fireWith:function(a,c){return!h||d&&!i||(c=c||[],c=[a,c.slice?c.slice():c],b?i.push(c):j(c)),this},fire:function(){return k.fireWith(this,arguments),this},fired:function(){return!!d}};return k},m.extend({Deferred:function(a){var b=[["resolve","done",m.Callbacks("once memory"),"resolved"],["reject","fail",m.Callbacks("once memory"),"rejected"],["notify","progress",m.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return m.Deferred(function(c){m.each(b,function(b,f){var g=m.isFunction(a[b])&&a[b];e[f[1]](function(){var a=g&&g.apply(this,arguments);a&&m.isFunction(a.promise)?a.promise().done(c.resolve).fail(c.reject).progress(c.notify):c[f[0]+"With"](this===d?c.promise():this,g?[a]:arguments)})}),a=null}).promise()},promise:function(a){return null!=a?m.extend(a,d):d}},e={};return d.pipe=d.then,m.each(b,function(a,f){var g=f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[1^a][2].disable,b[2][2].lock),e[f[0]]=function(){return e[f[0]+"With"](this===e?d:this,arguments),this},e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=d.call(arguments),e=c.length,f=1!==e||a&&m.isFunction(a.promise)?e:0,g=1===f?a:m.Deferred(),h=function(a,b,c){return function(e){b[a]=this,c[a]=arguments.length>1?d.call(arguments):e,c===i?g.notifyWith(b,c):--f||g.resolveWith(b,c)}},i,j,k;if(e>1)for(i=new Array(e),j=new Array(e),k=new Array(e);e>b;b++)c[b]&&m.isFunction(c[b].promise)?c[b].promise().done(h(b,k,c)).fail(g.reject).progress(h(b,j,i)):--f;return f||g.resolveWith(k,c),g.promise()}});var H;m.fn.ready=function(a){return m.ready.promise().done(a),this},m.extend({isReady:!1,readyWait:1,holdReady:function(a){a?m.readyWait++:m.ready(!0)},ready:function(a){if(a===!0?!--m.readyWait:!m.isReady){if(!y.body)return setTimeout(m.ready);m.isReady=!0,a!==!0&&--m.readyWait>0||(H.resolveWith(y,[m]),m.fn.triggerHandler&&(m(y).triggerHandler("ready"),m(y).off("ready")))}}});function I(){y.addEventListener?(y.removeEventListener("DOMContentLoaded",J,!1),a.removeEventListener("load",J,!1)):(y.detachEvent("onreadystatechange",J),a.detachEvent("onload",J))}function J(){(y.addEventListener||"load"===event.type||"complete"===y.readyState)&&(I(),m.ready())}m.ready.promise=function(b){if(!H)if(H=m.Deferred(),"complete"===y.readyState)setTimeout(m.ready);else if(y.addEventListener)y.addEventListener("DOMContentLoaded",J,!1),a.addEventListener("load",J,!1);else{y.attachEvent("onreadystatechange",J),a.attachEvent("onload",J);var c=!1;try{c=null==a.frameElement&&y.documentElement}catch(d){}c&&c.doScroll&&!function e(){if(!m.isReady){try{c.doScroll("left")}catch(a){return setTimeout(e,50)}I(),m.ready()}}()}return H.promise(b)};var K="undefined",L;for(L in m(k))break;k.ownLast="0"!==L,k.inlineBlockNeedsLayout=!1,m(function(){var a,b,c,d;c=y.getElementsByTagName("body")[0],c&&c.style&&(b=y.createElement("div"),d=y.createElement("div"),d.style.cssText="position:absolute;border:0;width:0;height:0;top:0;left:-9999px",c.appendChild(d).appendChild(b),typeof b.style.zoom!==K&&(b.style.cssText="display:inline;margin:0;border:0;padding:1px;width:1px;zoom:1",k.inlineBlockNeedsLayout=a=3===b.offsetWidth,a&&(c.style.zoom=1)),c.removeChild(d))}),function(){var a=y.createElement("div");if(null==k.deleteExpando){k.deleteExpando=!0;try{delete a.test}catch(b){k.deleteExpando=!1}}a=null}(),m.acceptData=function(a){var b=m.noData[(a.nodeName+" ").toLowerCase()],c=+a.nodeType||1;return 1!==c&&9!==c?!1:!b||b!==!0&&a.getAttribute("classid")===b};var M=/^(?:\{[\w\W]*\}|\[[\w\W]*\])$/,N=/([A-Z])/g;function O(a,b,c){if(void 0===c&&1===a.nodeType){var d="data-"+b.replace(N,"-$1").toLowerCase();if(c=a.getAttribute(d),"string"==typeof c){try{c="true"===c?!0:"false"===c?!1:"null"===c?null:+c+""===c?+c:M.test(c)?m.parseJSON(c):c}catch(e){}m.data(a,b,c)}else c=void 0}return c}function P(a){var b;for(b in a)if(("data"!==b||!m.isEmptyObject(a[b]))&&"toJSON"!==b)return!1;return!0}function Q(a,b,d,e){if(m.acceptData(a)){var f,g,h=m.expando,i=a.nodeType,j=i?m.cache:a,k=i?a[h]:a[h]&&h; +if(k&&j[k]&&(e||j[k].data)||void 0!==d||"string"!=typeof b)return k||(k=i?a[h]=c.pop()||m.guid++:h),j[k]||(j[k]=i?{}:{toJSON:m.noop}),("object"==typeof b||"function"==typeof b)&&(e?j[k]=m.extend(j[k],b):j[k].data=m.extend(j[k].data,b)),g=j[k],e||(g.data||(g.data={}),g=g.data),void 0!==d&&(g[m.camelCase(b)]=d),"string"==typeof b?(f=g[b],null==f&&(f=g[m.camelCase(b)])):f=g,f}}function R(a,b,c){if(m.acceptData(a)){var d,e,f=a.nodeType,g=f?m.cache:a,h=f?a[m.expando]:m.expando;if(g[h]){if(b&&(d=c?g[h]:g[h].data)){m.isArray(b)?b=b.concat(m.map(b,m.camelCase)):b in d?b=[b]:(b=m.camelCase(b),b=b in d?[b]:b.split(" ")),e=b.length;while(e--)delete d[b[e]];if(c?!P(d):!m.isEmptyObject(d))return}(c||(delete g[h].data,P(g[h])))&&(f?m.cleanData([a],!0):k.deleteExpando||g!=g.window?delete g[h]:g[h]=null)}}}m.extend({cache:{},noData:{"applet ":!0,"embed ":!0,"object ":"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000"},hasData:function(a){return a=a.nodeType?m.cache[a[m.expando]]:a[m.expando],!!a&&!P(a)},data:function(a,b,c){return Q(a,b,c)},removeData:function(a,b){return R(a,b)},_data:function(a,b,c){return Q(a,b,c,!0)},_removeData:function(a,b){return R(a,b,!0)}}),m.fn.extend({data:function(a,b){var c,d,e,f=this[0],g=f&&f.attributes;if(void 0===a){if(this.length&&(e=m.data(f),1===f.nodeType&&!m._data(f,"parsedAttrs"))){c=g.length;while(c--)g[c]&&(d=g[c].name,0===d.indexOf("data-")&&(d=m.camelCase(d.slice(5)),O(f,d,e[d])));m._data(f,"parsedAttrs",!0)}return e}return"object"==typeof a?this.each(function(){m.data(this,a)}):arguments.length>1?this.each(function(){m.data(this,a,b)}):f?O(f,a,m.data(f,a)):void 0},removeData:function(a){return this.each(function(){m.removeData(this,a)})}}),m.extend({queue:function(a,b,c){var d;return a?(b=(b||"fx")+"queue",d=m._data(a,b),c&&(!d||m.isArray(c)?d=m._data(a,b,m.makeArray(c)):d.push(c)),d||[]):void 0},dequeue:function(a,b){b=b||"fx";var c=m.queue(a,b),d=c.length,e=c.shift(),f=m._queueHooks(a,b),g=function(){m.dequeue(a,b)};"inprogress"===e&&(e=c.shift(),d--),e&&("fx"===b&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return m._data(a,c)||m._data(a,c,{empty:m.Callbacks("once memory").add(function(){m._removeData(a,b+"queue"),m._removeData(a,c)})})}}),m.fn.extend({queue:function(a,b){var c=2;return"string"!=typeof a&&(b=a,a="fx",c--),arguments.lengthh;h++)b(a[h],c,g?d:d.call(a[h],h,b(a[h],c)));return e?a:j?b.call(a):i?b(a[0],c):f},W=/^(?:checkbox|radio)$/i;!function(){var a=y.createElement("input"),b=y.createElement("div"),c=y.createDocumentFragment();if(b.innerHTML="
    Property NameDefaultMeaning
    spark.history.updateIntervalspark.history.providerorg.apache.spark.deploy.history.FsHistoryProviderName of the class implementing the application history backend. Currently there is only + one implementation, provided by Spark, which looks for application logs stored in the + file system.
    spark.history.fs.updateInterval 10 The period, in seconds, at which information displayed by this history server is updated. @@ -78,7 +87,7 @@ represents an application's event logs. This creates a web interface at
    spark.history.retainedApplications25050 The number of application UIs to retain. If this cap is exceeded, then the oldest applications will be removed. diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index af620d61607fa..1621833e124f5 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -34,7 +34,10 @@ object MimaExcludes { val excludes = SparkBuild.SPARK_VERSION match { case v if v.startsWith("1.1") => - Seq(MimaBuild.excludeSparkPackage("graphx")) ++ + Seq( + MimaBuild.excludeSparkPackage("deploy"), + MimaBuild.excludeSparkPackage("graphx") + ) ++ Seq( // Adding new method to JavaRDLike trait - we should probably mark this as a developer API. ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitions"), diff --git a/sbin/start-history-server.sh b/sbin/start-history-server.sh index 4a90c68763b68..e30493da32a7a 100755 --- a/sbin/start-history-server.sh +++ b/sbin/start-history-server.sh @@ -19,19 +19,18 @@ # Starts the history server on the machine this script is executed on. # -# Usage: start-history-server.sh [] -# Example: ./start-history-server.sh --dir /tmp/spark-events --port 18080 +# Usage: start-history-server.sh +# +# Use the SPARK_HISTORY_OPTS environment variable to set history server configuration. # sbin=`dirname "$0"` sbin=`cd "$sbin"; pwd` -if [ $# -lt 1 ]; then - echo "Usage: ./start-history-server.sh " - echo "Example: ./start-history-server.sh /tmp/spark-events" - exit +if [ $# != 0 ]; then + echo "Using command line arguments for setting the log directory is deprecated. Please " + echo "set the spark.history.fs.logDirectory configuration option instead." + export SPARK_HISTORY_OPTS="$SPARK_HISTORY_OPTS -Dspark.history.fs.logDirectory=$1" fi -LOG_DIR=$1 - -"$sbin"/spark-daemon.sh start org.apache.spark.deploy.history.HistoryServer 1 --dir "$LOG_DIR" +exec "$sbin"/spark-daemon.sh start org.apache.spark.deploy.history.HistoryServer 1 From 383bf72c115b09d68cbde0d35ed89808ce04863d Mon Sep 17 00:00:00 2001 From: Henry Saputra Date: Mon, 23 Jun 2014 17:13:26 -0700 Subject: [PATCH 027/124] Cleanup on Connection, ConnectionManagerId, ConnectionManager classes part 2 Cleanup on Connection, ConnectionManagerId, and ConnectionManager classes part 2 while I was working at the code there to help IDE: 1. Remove unused imports 2. Remove parentheses in method calls that do not have side affect. 3. Add parentheses in method calls that do have side effect or not simple get to object properties. 4. Change if-else check (via isInstanceOf) for Connection class type with Scala expression for consistency and cleanliness. 5. Remove semicolon 6. Remove extra spaces. 7. Remove redundant return for consistency Author: Henry Saputra Closes #1157 from hsaputra/cleanup_connection_classes_part2 and squashes the following commits: 4be6906 [Henry Saputra] Fix Spark Scala style for line over 100 chars. 85b24f7 [Henry Saputra] Cleanup on Connection and ConnectionManager classes part 2 while I was working at the code there to help IDE: 1. Remove unused imports 2. Remove parentheses in method calls that do not have side affect. 3. Add parentheses in method calls that do have side effect. 4. Change if-else check (via isInstanceOf) for Connection class type with Scala expression for consitency and cleanliness. 5. Remove semicolon 6. Remove extra spaces. --- .../org/apache/spark/network/Connection.scala | 15 +-- .../spark/network/ConnectionManager.scala | 114 +++++++++--------- .../spark/network/ConnectionManagerId.scala | 2 +- 3 files changed, 62 insertions(+), 69 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/Connection.scala index 3b6298a26d7c5..5285ec82c1b64 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/Connection.scala @@ -17,11 +17,6 @@ package org.apache.spark.network -import org.apache.spark._ -import org.apache.spark.SparkSaslServer - -import scala.collection.mutable.{HashMap, Queue, ArrayBuffer} - import java.net._ import java.nio._ import java.nio.channels._ @@ -41,7 +36,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, def this(channel_ : SocketChannel, selector_ : Selector, id_ : ConnectionId) = { this(channel_, selector_, ConnectionManagerId.fromSocketAddress( - channel_.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress]), id_) + channel_.socket.getRemoteSocketAddress.asInstanceOf[InetSocketAddress]), id_) } channel.configureBlocking(false) @@ -89,7 +84,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, private def disposeSasl() { if (sparkSaslServer != null) { - sparkSaslServer.dispose(); + sparkSaslServer.dispose() } if (sparkSaslClient != null) { @@ -328,15 +323,13 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, // Is highly unlikely unless there was an unclean close of socket, etc registerInterest() logInfo("Connected to [" + address + "], " + outbox.messages.size + " messages pending") - true } catch { case e: Exception => { logWarning("Error finishing connection to " + address, e) callOnExceptionCallback(e) - // ignore - return true } } + true } override def write(): Boolean = { @@ -546,7 +539,7 @@ private[spark] class ReceivingConnection( /* println("Filled buffer at " + System.currentTimeMillis) */ val bufferMessage = inbox.getMessageForChunk(currentChunk).get if (bufferMessage.isCompletelyReceived) { - bufferMessage.flip + bufferMessage.flip() bufferMessage.finishTime = System.currentTimeMillis logDebug("Finished receiving [" + bufferMessage + "] from " + "[" + getRemoteConnectionManagerId() + "] in " + bufferMessage.timeTaken) 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 cf1c985c2fff9..8a1cdb812962e 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -249,7 +249,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, def run() { try { while(!selectorThread.isInterrupted) { - while (! registerRequests.isEmpty) { + while (!registerRequests.isEmpty) { val conn: SendingConnection = registerRequests.dequeue() addListeners(conn) conn.connect() @@ -308,7 +308,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, // Some keys within the selectors list are invalid/closed. clear them. val allKeys = selector.keys().iterator() - while (allKeys.hasNext()) { + while (allKeys.hasNext) { val key = allKeys.next() try { if (! key.isValid) { @@ -341,7 +341,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, if (0 != selectedKeysCount) { val selectedKeys = selector.selectedKeys().iterator() - while (selectedKeys.hasNext()) { + while (selectedKeys.hasNext) { val key = selectedKeys.next selectedKeys.remove() try { @@ -419,62 +419,63 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, connectionsByKey -= connection.key try { - if (connection.isInstanceOf[SendingConnection]) { - val sendingConnection = connection.asInstanceOf[SendingConnection] - val sendingConnectionManagerId = sendingConnection.getRemoteConnectionManagerId() - logInfo("Removing SendingConnection to " + sendingConnectionManagerId) - - connectionsById -= sendingConnectionManagerId - connectionsAwaitingSasl -= connection.connectionId + connection match { + case sendingConnection: SendingConnection => + val sendingConnectionManagerId = sendingConnection.getRemoteConnectionManagerId() + logInfo("Removing SendingConnection to " + sendingConnectionManagerId) + + connectionsById -= sendingConnectionManagerId + connectionsAwaitingSasl -= connection.connectionId + + messageStatuses.synchronized { + messageStatuses.values.filter(_.connectionManagerId == sendingConnectionManagerId) + .foreach(status => { + logInfo("Notifying " + status) + status.synchronized { + status.attempted = true + status.acked = false + status.markDone() + } + }) - messageStatuses.synchronized { - messageStatuses - .values.filter(_.connectionManagerId == sendingConnectionManagerId).foreach(status => { - logInfo("Notifying " + status) - status.synchronized { - status.attempted = true - status.acked = false - status.markDone() - } + messageStatuses.retain((i, status) => { + status.connectionManagerId != sendingConnectionManagerId }) + } + case receivingConnection: ReceivingConnection => + val remoteConnectionManagerId = receivingConnection.getRemoteConnectionManagerId() + logInfo("Removing ReceivingConnection to " + remoteConnectionManagerId) - messageStatuses.retain((i, status) => { - status.connectionManagerId != sendingConnectionManagerId - }) - } - } else if (connection.isInstanceOf[ReceivingConnection]) { - val receivingConnection = connection.asInstanceOf[ReceivingConnection] - val remoteConnectionManagerId = receivingConnection.getRemoteConnectionManagerId() - logInfo("Removing ReceivingConnection to " + remoteConnectionManagerId) - - val sendingConnectionOpt = connectionsById.get(remoteConnectionManagerId) - if (! sendingConnectionOpt.isDefined) { - logError("Corresponding SendingConnectionManagerId not found") - return - } + val sendingConnectionOpt = connectionsById.get(remoteConnectionManagerId) + if (!sendingConnectionOpt.isDefined) { + logError("Corresponding SendingConnectionManagerId not found") + return + } - val sendingConnection = sendingConnectionOpt.get - connectionsById -= remoteConnectionManagerId - sendingConnection.close() + val sendingConnection = sendingConnectionOpt.get + connectionsById -= remoteConnectionManagerId + sendingConnection.close() - val sendingConnectionManagerId = sendingConnection.getRemoteConnectionManagerId() + val sendingConnectionManagerId = sendingConnection.getRemoteConnectionManagerId() - assert (sendingConnectionManagerId == remoteConnectionManagerId) + assert(sendingConnectionManagerId == remoteConnectionManagerId) - messageStatuses.synchronized { - for (s <- messageStatuses.values if s.connectionManagerId == sendingConnectionManagerId) { - logInfo("Notifying " + s) - s.synchronized { - s.attempted = true - s.acked = false - s.markDone() + messageStatuses.synchronized { + for (s <- messageStatuses.values + if s.connectionManagerId == sendingConnectionManagerId) { + logInfo("Notifying " + s) + s.synchronized { + s.attempted = true + s.acked = false + s.markDone() + } } - } - messageStatuses.retain((i, status) => { - status.connectionManagerId != sendingConnectionManagerId - }) - } + messageStatuses.retain((i, status) => { + status.connectionManagerId != sendingConnectionManagerId + }) + } + case _ => logError("Unsupported type of connection.") } } finally { // So that the selection keys can be removed. @@ -517,13 +518,13 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, logDebug("Client sasl completed for id: " + waitingConn.connectionId) connectionsAwaitingSasl -= waitingConn.connectionId waitingConn.getAuthenticated().synchronized { - waitingConn.getAuthenticated().notifyAll(); + waitingConn.getAuthenticated().notifyAll() } return } else { var replyToken : Array[Byte] = null try { - replyToken = waitingConn.sparkSaslClient.saslResponse(securityMsg.getToken); + replyToken = waitingConn.sparkSaslClient.saslResponse(securityMsg.getToken) if (waitingConn.isSaslComplete()) { logDebug("Client sasl completed after evaluate for id: " + waitingConn.connectionId) connectionsAwaitingSasl -= waitingConn.connectionId @@ -533,7 +534,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, return } val securityMsgResp = SecurityMessage.fromResponse(replyToken, - securityMsg.getConnectionId.toString()) + securityMsg.getConnectionId.toString) val message = securityMsgResp.toBufferMessage if (message == null) throw new Exception("Error creating security message") sendSecurityMessage(waitingConn.getRemoteConnectionManagerId(), message) @@ -630,13 +631,13 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, case bufferMessage: BufferMessage => { if (authEnabled) { val res = handleAuthentication(connection, bufferMessage) - if (res == true) { + if (res) { // message was security negotiation so skip the rest logDebug("After handleAuth result was true, returning") return } } - if (bufferMessage.hasAckId) { + if (bufferMessage.hasAckId()) { val sentMessageStatus = messageStatuses.synchronized { messageStatuses.get(bufferMessage.ackId) match { case Some(status) => { @@ -646,7 +647,6 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, case None => { throw new Exception("Could not find reference for received ack message " + message.id) - null } } } @@ -668,7 +668,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, if (ackMessage.isDefined) { if (!ackMessage.get.isInstanceOf[BufferMessage]) { logDebug("Response to " + bufferMessage + " is not a buffer message, it is of type " - + ackMessage.get.getClass()) + + ackMessage.get.getClass) } else if (!ackMessage.get.asInstanceOf[BufferMessage].hasAckId) { logDebug("Response to " + bufferMessage + " does not have ack id set") ackMessage.get.asInstanceOf[BufferMessage].ackId = bufferMessage.id diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala index b82edb6850d23..57f7586883af1 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala @@ -32,6 +32,6 @@ private[spark] case class ConnectionManagerId(host: String, port: Int) { private[spark] object ConnectionManagerId { def fromSocketAddress(socketAddress: InetSocketAddress): ConnectionManagerId = { - new ConnectionManagerId(socketAddress.getHostName(), socketAddress.getPort()) + new ConnectionManagerId(socketAddress.getHostName, socketAddress.getPort) } } From 51c8168377a89d20d0b2d7b9a28af58593a0fe0c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 23 Jun 2014 18:34:54 -0700 Subject: [PATCH 028/124] [SPARK-2227] Support dfs command in SQL. Note that nothing gets printed to the console because we don't properly maintain session state right now. I will have a followup PR that fixes it. Author: Reynold Xin Closes #1167 from rxin/commands and squashes the following commits: 56f04f8 [Reynold Xin] [SPARK-2227] Support dfs command in SQL. --- .../scala/org/apache/spark/sql/hive/HiveQl.scala | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) 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 b073dc3895f05..b70104dd5be5a 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 @@ -38,8 +38,6 @@ import scala.collection.JavaConversions._ */ private[hive] case object NativePlaceholder extends Command -private[hive] case class DfsCommand(cmd: String) extends Command - private[hive] case class ShellCommand(cmd: String) extends Command private[hive] case class SourceCommand(filePath: String) extends Command @@ -227,15 +225,15 @@ private[hive] object HiveQl { SetCommand(Some(key), Some(value)) } } else if (sql.trim.toLowerCase.startsWith("cache table")) { - CacheCommand(sql.drop(12).trim, true) + CacheCommand(sql.trim.drop(12).trim, true) } else if (sql.trim.toLowerCase.startsWith("uncache table")) { - CacheCommand(sql.drop(14).trim, false) + CacheCommand(sql.trim.drop(14).trim, false) } else if (sql.trim.toLowerCase.startsWith("add jar")) { - AddJar(sql.drop(8)) + AddJar(sql.trim.drop(8)) } else if (sql.trim.toLowerCase.startsWith("add file")) { - AddFile(sql.drop(9)) - } else if (sql.trim.startsWith("dfs")) { - DfsCommand(sql) + AddFile(sql.trim.drop(9)) + } else if (sql.trim.toLowerCase.startsWith("dfs")) { + NativeCommand(sql) } else if (sql.trim.startsWith("source")) { SourceCommand(sql.split(" ").toSeq match { case Seq("source", filePath) => filePath }) } else if (sql.trim.startsWith("!")) { From 56eb8af187b19f09810baafb314b21e07cf0a79c Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 23 Jun 2014 20:25:46 -0700 Subject: [PATCH 029/124] [SPARK-2124] Move aggregation into shuffle implementations This PR is a sub-task of SPARK-2044 to move the execution of aggregation into shuffle implementations. I leave `CoGoupedRDD` and `SubtractedRDD` unchanged because they have their implementations of aggregation. I'm not sure is it suitable to change these two RDDs. Also I do not move sort related code of `OrderedRDDFunctions` into shuffle, this will be solved in another sub-task. Author: jerryshao Closes #1064 from jerryshao/SPARK-2124 and squashes the following commits: 4a05a40 [jerryshao] Modify according to comments 1f7dcc8 [jerryshao] Style changes 50a2fd6 [jerryshao] Fix test suite issue after moving aggregator to Shuffle reader and writer 1a96190 [jerryshao] Code modification related to the ShuffledRDD 308f635 [jerryshao] initial works of move combiner to ShuffleManager's reader and writer --- .../scala/org/apache/spark/Dependency.scala | 3 +- .../spark/rdd/OrderedRDDFunctions.scala | 2 +- .../apache/spark/rdd/PairRDDFunctions.scala | 20 +++------- .../main/scala/org/apache/spark/rdd/RDD.scala | 2 +- .../org/apache/spark/rdd/ShuffledRDD.scala | 37 ++++++++++++++++--- .../spark/scheduler/ShuffleMapTask.scala | 6 +-- .../apache/spark/shuffle/ShuffleWriter.scala | 4 +- .../shuffle/hash/HashShuffleReader.scala | 20 ++++++++-- .../shuffle/hash/HashShuffleWriter.scala | 23 +++++++++--- .../org/apache/spark/CheckpointSuite.scala | 2 +- .../scala/org/apache/spark/ShuffleSuite.scala | 22 +++++++---- .../scala/org/apache/spark/rdd/RDDSuite.scala | 14 +++---- .../spark/scheduler/SparkListenerSuite.scala | 2 +- .../graphx/impl/MessageToPartition.scala | 7 ++-- .../graphx/impl/RoutingTablePartition.scala | 4 +- .../apache/spark/sql/execution/Exchange.scala | 6 +-- .../spark/sql/execution/basicOperators.scala | 2 +- 17 files changed, 112 insertions(+), 64 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index c8c194a111aac..09a60571238ea 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -61,7 +61,8 @@ class ShuffleDependency[K, V, C]( val partitioner: Partitioner, val serializer: Option[Serializer] = None, val keyOrdering: Option[Ordering[K]] = None, - val aggregator: Option[Aggregator[K, V, C]] = None) + val aggregator: Option[Aggregator[K, V, C]] = None, + val mapSideCombine: Boolean = false) extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) { val shuffleId: Int = rdd.context.newShuffleId() 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 6a3f698444283..f1f4b4324edfd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -57,7 +57,7 @@ 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, P](self, part) + val shuffled = new ShuffledRDD[K, V, V, P](self, part).setKeyOrdering(ordering) shuffled.mapPartitions(iter => { val buf = iter.toArray if (ascending) { diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 443d1c587c3ee..2b2b9ae3fd533 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -90,21 +90,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) self.mapPartitionsWithContext((context, iter) => { new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context)) }, preservesPartitioning = true) - } else if (mapSideCombine) { - val combined = self.mapPartitionsWithContext((context, iter) => { - aggregator.combineValuesByKey(iter, context) - }, preservesPartitioning = true) - val partitioned = new ShuffledRDD[K, C, (K, C)](combined, partitioner) - .setSerializer(serializer) - partitioned.mapPartitionsWithContext((context, iter) => { - new InterruptibleIterator(context, aggregator.combineCombinersByKey(iter, context)) - }, preservesPartitioning = true) } else { - // Don't apply map-side combiner. - val values = new ShuffledRDD[K, V, (K, V)](self, partitioner).setSerializer(serializer) - values.mapPartitionsWithContext((context, iter) => { - new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context)) - }, preservesPartitioning = true) + new ShuffledRDD[K, V, C, (K, C)](self, partitioner) + .setSerializer(serializer) + .setAggregator(aggregator) + .setMapSideCombine(mapSideCombine) } } @@ -401,7 +391,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) if (self.partitioner == Some(partitioner)) { self } else { - new ShuffledRDD[K, V, (K, V)](self, partitioner) + new ShuffledRDD[K, V, V, (K, V)](self, partitioner) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index cebfd109d825f..4e841bc992bff 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -340,7 +340,7 @@ abstract class RDD[T: ClassTag]( // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( - new ShuffledRDD[Int, T, (Int, T)](mapPartitionsWithIndex(distributePartition), + new ShuffledRDD[Int, T, T, (Int, T)](mapPartitionsWithIndex(distributePartition), new HashPartitioner(numPartitions)), numPartitions).values } else { 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 bb108ef163c56..bf02f68d0d3d3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency, SparkEnv, TaskContext} +import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.serializer.Serializer @@ -35,23 +35,48 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { * @param part the partitioner used to partition the RDD * @tparam K the key class. * @tparam V the value class. + * @tparam C the combiner class. */ @DeveloperApi -class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( - @transient var prev: RDD[P], +class ShuffledRDD[K, V, C, P <: Product2[K, C] : ClassTag]( + @transient var prev: RDD[_ <: Product2[K, V]], part: Partitioner) extends RDD[P](prev.context, Nil) { private var serializer: Option[Serializer] = None + private var keyOrdering: Option[Ordering[K]] = None + + private var aggregator: Option[Aggregator[K, V, C]] = None + + private var mapSideCombine: Boolean = false + /** Set a serializer for this RDD's shuffle, or null to use the default (spark.serializer) */ - def setSerializer(serializer: Serializer): ShuffledRDD[K, V, P] = { + def setSerializer(serializer: Serializer): ShuffledRDD[K, V, C, P] = { this.serializer = Option(serializer) this } + /** Set key ordering for RDD's shuffle. */ + def setKeyOrdering(keyOrdering: Ordering[K]): ShuffledRDD[K, V, C, P] = { + this.keyOrdering = Option(keyOrdering) + this + } + + /** Set aggregator for RDD's shuffle. */ + def setAggregator(aggregator: Aggregator[K, V, C]): ShuffledRDD[K, V, C, P] = { + this.aggregator = Option(aggregator) + this + } + + /** Set mapSideCombine flag for RDD's shuffle. */ + def setMapSideCombine(mapSideCombine: Boolean): ShuffledRDD[K, V, C, P] = { + this.mapSideCombine = mapSideCombine + this + } + override def getDependencies: Seq[Dependency[_]] = { - List(new ShuffleDependency(prev, part, serializer)) + List(new ShuffleDependency(prev, part, serializer, keyOrdering, aggregator, mapSideCombine)) } override val partitioner = Some(part) @@ -61,7 +86,7 @@ class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( } override def compute(split: Partition, context: TaskContext): Iterator[P] = { - val dep = dependencies.head.asInstanceOf[ShuffleDependency[K, V, V]] + val dep = dependencies.head.asInstanceOf[ShuffleDependency[K, V, C]] SparkEnv.get.shuffleManager.getReader(dep.shuffleHandle, split.index, split.index + 1, context) .read() .asInstanceOf[Iterator[P]] diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 859cdc524a581..fdaf1de83f051 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -144,10 +144,8 @@ private[spark] class ShuffleMapTask( try { val manager = SparkEnv.get.shuffleManager writer = manager.getWriter[Any, Any](dep.shuffleHandle, partitionId, context) - for (elem <- rdd.iterator(split, context)) { - writer.write(elem.asInstanceOf[Product2[Any, Any]]) - } - writer.stop(success = true).get + writer.write(rdd.iterator(split, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) + return writer.stop(success = true).get } catch { case e: Exception => if (writer != null) { diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index ead3ebd652ca5..b934480cfb9be 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -23,8 +23,8 @@ import org.apache.spark.scheduler.MapStatus * Obtained inside a map task to write out records to the shuffle system. */ private[spark] trait ShuffleWriter[K, V] { - /** Write a record to this task's output */ - def write(record: Product2[K, V]): Unit + /** Write a bunch of records to this task's output */ + def write(records: Iterator[_ <: Product2[K, V]]): Unit /** Close this writer, passing along whether the map completed */ def stop(success: Boolean): Option[MapStatus] 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 f6a790309a587..d45258c0a492b 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 @@ -17,9 +17,9 @@ package org.apache.spark.shuffle.hash +import org.apache.spark.{InterruptibleIterator, TaskContext} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleReader} -import org.apache.spark.TaskContext class HashShuffleReader[K, C]( handle: BaseShuffleHandle[K, _, C], @@ -31,10 +31,24 @@ class HashShuffleReader[K, C]( require(endPartition == startPartition + 1, "Hash shuffle currently only supports fetching one partition") + private val dep = handle.dependency + /** Read the combined key-values for this reduce task */ override def read(): Iterator[Product2[K, C]] = { - BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, - Serializer.getSerializer(handle.dependency.serializer)) + val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, + Serializer.getSerializer(dep.serializer)) + + if (dep.aggregator.isDefined) { + if (dep.mapSideCombine) { + new InterruptibleIterator(context, dep.aggregator.get.combineCombinersByKey(iter, context)) + } else { + new InterruptibleIterator(context, dep.aggregator.get.combineValuesByKey(iter, context)) + } + } else if (dep.aggregator.isEmpty && dep.mapSideCombine) { + throw new IllegalStateException("Aggregator is empty for map-side combine") + } else { + iter + } } /** Close this reader */ diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 4c6749098c110..9b78228519da4 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -40,11 +40,24 @@ class HashShuffleWriter[K, V]( private val ser = Serializer.getSerializer(dep.serializer.getOrElse(null)) private val shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, mapId, numOutputSplits, ser) - /** Write a record to this task's output */ - override def write(record: Product2[K, V]): Unit = { - val pair = record.asInstanceOf[Product2[Any, Any]] - val bucketId = dep.partitioner.getPartition(pair._1) - shuffle.writers(bucketId).write(pair) + /** Write a bunch of records to this task's output */ + override def write(records: Iterator[_ <: Product2[K, V]]): Unit = { + val iter = if (dep.aggregator.isDefined) { + if (dep.mapSideCombine) { + dep.aggregator.get.combineValuesByKey(records, context) + } else { + records + } + } else if (dep.aggregator.isEmpty && dep.mapSideCombine) { + throw new IllegalStateException("Aggregator is empty for map-side combine") + } else { + records + } + + for (elem <- iter) { + val bucketId = dep.partitioner.getPartition(elem._1) + shuffle.writers(bucketId).write(elem) + } } /** Close this writer, passing along whether the map completed */ diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index f64f3c9036034..fc00458083a33 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -99,7 +99,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { test("ShuffledRDD") { testRDD(rdd => { // Creating ShuffledRDD directly as PairRDDFunctions.combineByKey produces a MapPartitionedRDD - new ShuffledRDD[Int, Int, (Int, Int)](rdd.map(x => (x % 2, 1)), partitioner) + new ShuffledRDD[Int, Int, Int, (Int, Int)](rdd.map(x => (x % 2, 1)), partitioner) }) } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 47112ce66d695..b40fee7e9ab23 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -56,8 +56,11 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { } // If the Kryo serializer is not used correctly, the shuffle would fail because the // default Java serializer cannot handle the non serializable class. - val c = new ShuffledRDD[Int, NonJavaSerializableClass, (Int, NonJavaSerializableClass)]( - b, new HashPartitioner(NUM_BLOCKS)).setSerializer(new KryoSerializer(conf)) + val c = new ShuffledRDD[Int, + NonJavaSerializableClass, + NonJavaSerializableClass, + (Int, NonJavaSerializableClass)](b, new HashPartitioner(NUM_BLOCKS)) + c.setSerializer(new KryoSerializer(conf)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId assert(c.count === 10) @@ -78,8 +81,11 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { } // If the Kryo serializer is not used correctly, the shuffle would fail because the // default Java serializer cannot handle the non serializable class. - val c = new ShuffledRDD[Int, NonJavaSerializableClass, (Int, NonJavaSerializableClass)]( - b, new HashPartitioner(3)).setSerializer(new KryoSerializer(conf)) + val c = new ShuffledRDD[Int, + NonJavaSerializableClass, + NonJavaSerializableClass, + (Int, NonJavaSerializableClass)](b, new HashPartitioner(3)) + c.setSerializer(new KryoSerializer(conf)) assert(c.count === 10) } @@ -94,7 +100,7 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { // NOTE: The default Java serializer doesn't create zero-sized blocks. // So, use Kryo - val c = new ShuffledRDD[Int, Int, (Int, Int)](b, new HashPartitioner(10)) + val c = new ShuffledRDD[Int, Int, Int, (Int, Int)](b, new HashPartitioner(10)) .setSerializer(new KryoSerializer(conf)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId @@ -120,7 +126,7 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { val b = a.map(x => (x, x*2)) // NOTE: The default Java serializer should create zero-sized blocks - val c = new ShuffledRDD[Int, Int, (Int, Int)](b, new HashPartitioner(10)) + val c = new ShuffledRDD[Int, Int, Int, (Int, Int)](b, new HashPartitioner(10)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId assert(c.count === 4) @@ -141,8 +147,8 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) val data = Array(p(1, 1), p(1, 2), p(1, 3), p(2, 1)) val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2) - val results = new ShuffledRDD[Int, Int, MutablePair[Int, Int]](pairs, new HashPartitioner(2)) - .collect() + val results = new ShuffledRDD[Int, Int, Int, MutablePair[Int, Int]](pairs, + new HashPartitioner(2)).collect() data.foreach { pair => results should contain (pair) } } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 0e5625b7645d5..0f9cbe213ea17 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -276,7 +276,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { // we can optionally shuffle to keep the upstream parallel val coalesced5 = data.coalesce(1, shuffle = true) val isEquals = coalesced5.dependencies.head.rdd.dependencies.head.rdd. - asInstanceOf[ShuffledRDD[_, _, _]] != null + asInstanceOf[ShuffledRDD[_, _, _, _]] != null assert(isEquals) // when shuffling, we can increase the number of partitions @@ -509,7 +509,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("takeSample") { val n = 1000000 val data = sc.parallelize(1 to n, 2) - + for (num <- List(5, 20, 100)) { val sample = data.takeSample(withReplacement=false, num=num) assert(sample.size === num) // Got exactly num elements @@ -704,11 +704,11 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(ancestors3.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) // Any ancestors before the shuffle are not considered - assert(ancestors4.size === 1) - assert(ancestors4.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 1) - assert(ancestors5.size === 4) - assert(ancestors5.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 1) - assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 1) + assert(ancestors4.size === 0) + assert(ancestors4.count(_.isInstanceOf[ShuffledRDD[_, _, _, _]]) === 0) + assert(ancestors5.size === 3) + assert(ancestors5.count(_.isInstanceOf[ShuffledRDD[_, _, _, _]]) === 1) + assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 0) assert(ancestors5.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 2) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index abd7b22310f1a..6df0a080961b6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -181,7 +181,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {2} // Shuffle map stage + result stage val stageInfo3 = listener.stageInfos.keys.find(_.stageId == 2).get - stageInfo3.rddInfos.size should be {2} // ShuffledRDD, MapPartitionsRDD + stageInfo3.rddInfos.size should be {1} // ShuffledRDD stageInfo3.rddInfos.forall(_.numPartitions == 4) should be {true} stageInfo3.rddInfos.exists(_.name == "Trois") should be {true} } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala index 1c6d7e59e9a27..d85afa45b1264 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -62,7 +62,8 @@ class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/* , AnyRe private[graphx] class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T]]) { def partitionBy(partitioner: Partitioner): RDD[VertexBroadcastMsg[T]] = { - val rdd = new ShuffledRDD[PartitionID, (VertexId, T), VertexBroadcastMsg[T]](self, partitioner) + val rdd = new ShuffledRDD[PartitionID, (VertexId, T), (VertexId, T), VertexBroadcastMsg[T]]( + self, partitioner) // Set a custom serializer if the data is of int or double type. if (classTag[T] == ClassTag.Int) { @@ -84,7 +85,7 @@ class MsgRDDFunctions[T: ClassTag](self: RDD[MessageToPartition[T]]) { * Return a copy of the RDD partitioned using the specified partitioner. */ def partitionBy(partitioner: Partitioner): RDD[MessageToPartition[T]] = { - new ShuffledRDD[PartitionID, T, MessageToPartition[T]](self, partitioner) + new ShuffledRDD[PartitionID, T, T, MessageToPartition[T]](self, partitioner) } } @@ -103,7 +104,7 @@ object MsgRDDFunctions { private[graphx] class VertexRDDFunctions[VD: ClassTag](self: RDD[(VertexId, VD)]) { def copartitionWithVertices(partitioner: Partitioner): RDD[(VertexId, VD)] = { - val rdd = new ShuffledRDD[VertexId, VD, (VertexId, VD)](self, partitioner) + val rdd = new ShuffledRDD[VertexId, VD, VD, (VertexId, VD)](self, partitioner) // Set a custom serializer if the data is of int or double type. if (classTag[VD] == ClassTag.Int) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index d02e9238adba5..3827ac8d0fd6a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -46,8 +46,8 @@ private[graphx] class RoutingTableMessageRDDFunctions(self: RDD[RoutingTableMessage]) { /** Copartition an `RDD[RoutingTableMessage]` with the vertex RDD with the given `partitioner`. */ def copartitionWithVertices(partitioner: Partitioner): RDD[RoutingTableMessage] = { - new ShuffledRDD[VertexId, (PartitionID, Byte), RoutingTableMessage](self, partitioner) - .setSerializer(new RoutingTableMessageSerializer) + new ShuffledRDD[VertexId, (PartitionID, Byte), (PartitionID, Byte), RoutingTableMessage]( + self, partitioner).setSerializer(new RoutingTableMessageSerializer) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index f46fa0516566f..00010ef6e798a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -47,7 +47,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una iter.map(r => mutablePair.update(hashExpressions(r), r)) } val part = new HashPartitioner(numPartitions) - val shuffled = new ShuffledRDD[Row, Row, MutablePair[Row, Row]](rdd, part) + val shuffled = new ShuffledRDD[Row, Row, Row, MutablePair[Row, Row]](rdd, part) shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._2) @@ -60,7 +60,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una iter.map(row => mutablePair.update(row, null)) } val part = new RangePartitioner(numPartitions, rdd, ascending = true) - val shuffled = new ShuffledRDD[Row, Null, MutablePair[Row, Null]](rdd, part) + val shuffled = new ShuffledRDD[Row, Null, Null, MutablePair[Row, Null]](rdd, part) shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._1) @@ -71,7 +71,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una iter.map(r => mutablePair.update(null, r)) } val partitioner = new HashPartitioner(1) - val shuffled = new ShuffledRDD[Null, Row, MutablePair[Null, Row]](rdd, partitioner) + val shuffled = new ShuffledRDD[Null, Row, Row, MutablePair[Null, Row]](rdd, partitioner) shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._2) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 18f4a5877bb21..b40d4e3a3b777 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -105,7 +105,7 @@ case class Limit(limit: Int, child: SparkPlan)(@transient sqlContext: SQLContext iter.take(limit).map(row => mutablePair.update(false, row)) } val part = new HashPartitioner(1) - val shuffled = new ShuffledRDD[Boolean, Row, MutablePair[Boolean, Row]](rdd, part) + val shuffled = new ShuffledRDD[Boolean, Row, Row, MutablePair[Boolean, Row]](rdd, part) shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.mapPartitions(_.take(limit).map(_._2)) } From 420c1c3e1beea03453e0eb9dc06f226c80496d68 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 23 Jun 2014 23:18:47 -0700 Subject: [PATCH 030/124] [SPARK-2252] Fix MathJax for HTTPs. Found out about this from the Hacker News link to GraphX which was using HTTPs. @mengxr Author: Reynold Xin Closes #1189 from rxin/mllib-doc and squashes the following commits: 5328be0 [Reynold Xin] [SPARK-2252] Fix MathJax for HTTPs. --- docs/_layouts/global.html | 36 +++++++++++++++++++++++------------- 1 file changed, 23 insertions(+), 13 deletions(-) diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 4ba20e590f2c2..b30ab1e5218c0 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -136,21 +136,31 @@

    {{ page.title }}

    - + }); + From 924b7082b17b379e4b932a3afee88f94a632d14d Mon Sep 17 00:00:00 2001 From: Rui Li Date: Tue, 24 Jun 2014 11:40:37 -0700 Subject: [PATCH 031/124] SPARK-1937: fix issue with task locality Don't check executor/host availability when creating a TaskSetManager. Because the executors may haven't been registered when the TaskSetManager is created, in which case all tasks will be considered "has no preferred locations", and thus losing data locality in later scheduling. Author: Rui Li Author: lirui-intel Closes #892 from lirui-intel/delaySchedule and squashes the following commits: 8444d7c [Rui Li] fix code style fafd57f [Rui Li] keep locality constraints within the valid levels 18f9e05 [Rui Li] restrict allowed locality 5b3fb2f [Rui Li] refine UT 99f843e [Rui Li] add unit test and fix bug fff4123 [Rui Li] fix computing valid locality levels 685ed3d [Rui Li] remove delay shedule for pendingTasksWithNoPrefs 7b0177a [Rui Li] remove redundant code c7b93b5 [Rui Li] revise patch 3d7da02 [lirui-intel] Update TaskSchedulerImpl.scala cab4c71 [Rui Li] revised patch 539a578 [Rui Li] fix code style cf0d6ac [Rui Li] fix code style 3dfae86 [Rui Li] re-compute pending tasks when new host is added a225ac2 [Rui Li] SPARK-1937: fix issue with task locality --- .../spark/scheduler/TaskSchedulerImpl.scala | 8 +++- .../spark/scheduler/TaskSetManager.scala | 43 +++++++++++++------ .../spark/scheduler/TaskSetManagerSuite.scala | 34 +++++++++++++++ 3 files changed, 71 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 17292b4c15b8b..5ed2803d76afc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -210,11 +210,14 @@ private[spark] class TaskSchedulerImpl( SparkEnv.set(sc.env) // Mark each slave as alive and remember its hostname + // Also track if new executor is added + var newExecAvail = false for (o <- offers) { executorIdToHost(o.executorId) = o.host if (!executorsByHost.contains(o.host)) { executorsByHost(o.host) = new HashSet[String]() executorAdded(o.executorId, o.host) + newExecAvail = true } } @@ -227,12 +230,15 @@ private[spark] class TaskSchedulerImpl( for (taskSet <- sortedTaskSets) { logDebug("parentName: %s, name: %s, runningTasks: %s".format( taskSet.parent.name, taskSet.name, taskSet.runningTasks)) + if (newExecAvail) { + taskSet.executorAdded() + } } // Take each TaskSet in our scheduling order, and then offer it each node in increasing order // of locality levels so that it gets a chance to launch local tasks on all of them. var launchedTask = false - for (taskSet <- sortedTaskSets; maxLocality <- TaskLocality.values) { + for (taskSet <- sortedTaskSets; maxLocality <- taskSet.myLocalityLevels) { do { launchedTask = false for (i <- 0 until shuffledOffers.size) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index f3bd0797aa035..b5bcdd7e99c58 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -118,7 +118,7 @@ private[spark] class TaskSetManager( private val pendingTasksForRack = new HashMap[String, ArrayBuffer[Int]] // Set containing pending tasks with no locality preferences. - val pendingTasksWithNoPrefs = new ArrayBuffer[Int] + var pendingTasksWithNoPrefs = new ArrayBuffer[Int] // Set containing all pending tasks (also used as a stack, as above). val allPendingTasks = new ArrayBuffer[Int] @@ -153,8 +153,8 @@ private[spark] class TaskSetManager( } // Figure out which locality levels we have in our TaskSet, so we can do delay scheduling - val myLocalityLevels = computeValidLocalityLevels() - val localityWaits = myLocalityLevels.map(getLocalityWait) // Time to wait at each level + var myLocalityLevels = computeValidLocalityLevels() + var localityWaits = myLocalityLevels.map(getLocalityWait) // Time to wait at each level // Delay scheduling variables: we keep track of our current locality level and the time we // last launched a task at that level, and move up a level when localityWaits[curLevel] expires. @@ -181,16 +181,14 @@ private[spark] class TaskSetManager( var hadAliveLocations = false for (loc <- tasks(index).preferredLocations) { for (execId <- loc.executorId) { - if (sched.isExecutorAlive(execId)) { - addTo(pendingTasksForExecutor.getOrElseUpdate(execId, new ArrayBuffer)) - hadAliveLocations = true - } + addTo(pendingTasksForExecutor.getOrElseUpdate(execId, new ArrayBuffer)) } if (sched.hasExecutorsAliveOnHost(loc.host)) { - addTo(pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer)) - for (rack <- sched.getRackForHost(loc.host)) { - addTo(pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer)) - } + hadAliveLocations = true + } + addTo(pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer)) + for (rack <- sched.getRackForHost(loc.host)) { + addTo(pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer)) hadAliveLocations = true } } @@ -725,10 +723,12 @@ private[spark] class TaskSetManager( private def computeValidLocalityLevels(): Array[TaskLocality.TaskLocality] = { import TaskLocality.{PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY} val levels = new ArrayBuffer[TaskLocality.TaskLocality] - if (!pendingTasksForExecutor.isEmpty && getLocalityWait(PROCESS_LOCAL) != 0) { + if (!pendingTasksForExecutor.isEmpty && getLocalityWait(PROCESS_LOCAL) != 0 && + pendingTasksForExecutor.keySet.exists(sched.isExecutorAlive(_))) { levels += PROCESS_LOCAL } - if (!pendingTasksForHost.isEmpty && getLocalityWait(NODE_LOCAL) != 0) { + if (!pendingTasksForHost.isEmpty && getLocalityWait(NODE_LOCAL) != 0 && + pendingTasksForHost.keySet.exists(sched.hasExecutorsAliveOnHost(_))) { levels += NODE_LOCAL } if (!pendingTasksForRack.isEmpty && getLocalityWait(RACK_LOCAL) != 0) { @@ -738,4 +738,21 @@ private[spark] class TaskSetManager( logDebug("Valid locality levels for " + taskSet + ": " + levels.mkString(", ")) levels.toArray } + + // Re-compute pendingTasksWithNoPrefs since new preferred locations may become available + def executorAdded() { + def newLocAvail(index: Int): Boolean = { + for (loc <- tasks(index).preferredLocations) { + if (sched.hasExecutorsAliveOnHost(loc.host) || + sched.getRackForHost(loc.host).isDefined) { + return true + } + } + false + } + logInfo("Re-computing pending task lists.") + pendingTasksWithNoPrefs = pendingTasksWithNoPrefs.filter(!newLocAvail(_)) + myLocalityLevels = computeValidLocalityLevels() + localityWaits = myLocalityLevels.map(getLocalityWait) + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 6f1fd25764544..59a618956a356 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -77,6 +77,10 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex override def isExecutorAlive(execId: String): Boolean = executors.contains(execId) override def hasExecutorsAliveOnHost(host: String): Boolean = executors.values.exists(_ == host) + + def addExecutor(execId: String, host: String) { + executors.put(execId, host) + } } class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { @@ -400,6 +404,36 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(sched.taskSetsFailed.contains(taskSet.id)) } + test("new executors get added") { + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc) + val taskSet = FakeTask.createTaskSet(4, + Seq(TaskLocation("host1", "execA")), + Seq(TaskLocation("host1", "execB")), + Seq(TaskLocation("host2", "execC")), + Seq()) + val clock = new FakeClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + // All tasks added to no-pref list since no preferred location is available + assert(manager.pendingTasksWithNoPrefs.size === 4) + // Only ANY is valid + assert(manager.myLocalityLevels.sameElements(Array(ANY))) + // Add a new executor + sched.addExecutor("execD", "host1") + manager.executorAdded() + // Task 0 and 1 should be removed from no-pref list + assert(manager.pendingTasksWithNoPrefs.size === 2) + // Valid locality should contain NODE_LOCAL and ANY + assert(manager.myLocalityLevels.sameElements(Array(NODE_LOCAL, ANY))) + // Add another executor + sched.addExecutor("execC", "host2") + manager.executorAdded() + // No-pref list now only contains task 3 + assert(manager.pendingTasksWithNoPrefs.size === 1) + // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL and ANY + assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) + } + def createTaskResult(id: Int): DirectTaskResult[Int] = { val valueSer = SparkEnv.get.serializer.newInstance() new DirectTaskResult[Int](valueSer.serialize(id), mutable.Map.empty, new TaskMetrics) From 221909e678bb60e5a3746e64a718600841c5f74f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 24 Jun 2014 15:09:30 -0700 Subject: [PATCH 032/124] HOTFIX: Disabling tests per SPARK-2264 --- .../org/apache/spark/sql/CachedTableSuite.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index c794da4da4069..128ddae11eb0d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -24,7 +24,9 @@ import org.apache.spark.sql.test.TestSQLContext class CachedTableSuite extends QueryTest { TestData // Load test tables. - test("read from cached table and uncache") { + // NOTE: ALL TESTS ARE IGNORED PENDING SPARK-2264 + + ignore("read from cached table and uncache") { TestSQLContext.cacheTable("testData") checkAnswer( @@ -51,20 +53,20 @@ class CachedTableSuite extends QueryTest { } } - test("correct error on uncache of non-cached table") { + ignore("correct error on uncache of non-cached table") { intercept[IllegalArgumentException] { TestSQLContext.uncacheTable("testData") } } - test("SELECT Star Cached Table") { + ignore("SELECT Star Cached Table") { TestSQLContext.sql("SELECT * FROM testData").registerAsTable("selectStar") TestSQLContext.cacheTable("selectStar") TestSQLContext.sql("SELECT * FROM selectStar WHERE key = 1").collect() TestSQLContext.uncacheTable("selectStar") } - test("Self-join cached") { + ignore("Self-join cached") { val unCachedAnswer = TestSQLContext.sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key").collect() TestSQLContext.cacheTable("testData") @@ -74,7 +76,7 @@ class CachedTableSuite extends QueryTest { TestSQLContext.uncacheTable("testData") } - test("'CACHE TABLE' and 'UNCACHE TABLE' SQL statement") { + ignore("'CACHE TABLE' and 'UNCACHE TABLE' SQL statement") { TestSQLContext.sql("CACHE TABLE testData") TestSQLContext.table("testData").queryExecution.executedPlan match { case _: InMemoryColumnarTableScan => // Found evidence of caching From 1978a9033ea0d47ac0e0a51d97c8515689f84d04 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 24 Jun 2014 16:54:50 -0700 Subject: [PATCH 033/124] Fix broken Json tests. The assertJsonStringEquals method was missing an "assert" so did not actually check that the strings were equal. This commit adds the missing assert and fixes subsequently revealed problems with the JsonProtocolSuite. @andrewor14 I changed some of the test functionality to match what it looks like you intended based on the expected strings -- let me know if anything here looks wrong. Author: Kay Ousterhout Closes #1198 from kayousterhout/json_test_fix and squashes the following commits: 77f858f [Kay Ousterhout] Fix broken Json tests. --- .../apache/spark/util/JsonProtocolSuite.scala | 23 +++++++++---------- 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index f72389b6b323f..495e1b7a0a214 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -366,7 +366,7 @@ class JsonProtocolSuite extends FunSuite { private def assertJsonStringEquals(json1: String, json2: String) { val formatJsonString = (json: String) => json.replaceAll("[\\s|]", "") - formatJsonString(json1) === formatJsonString(json2) + assert(formatJsonString(json1) === formatJsonString(json2)) } private def assertSeqEquals[T](seq1: Seq[T], seq2: Seq[T], assertEquals: (T, T) => Unit) { @@ -449,7 +449,7 @@ class JsonProtocolSuite extends FunSuite { } private def makeStageInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { - val rddInfos = (1 to a % 5).map { i => makeRddInfo(a % i, b % i, c % i, d % i, e % i) } + val rddInfos = (0 until a % 5).map { i => makeRddInfo(a + i, b + i, c + i, d + i, e + i) } new StageInfo(a, "greetings", b, rddInfos, "details") } @@ -493,20 +493,19 @@ class JsonProtocolSuite extends FunSuite { private val stageSubmittedJsonString = """ {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":100,"Stage Name": - "greetings","Number of Tasks":200,"RDD Info":{"RDD ID":100,"Name":"mayor","Storage - Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":true, - "Replication":1},"Number of Partitions":200,"Number of Cached Partitions":300, - "Memory Size":400,"Disk Size":500,"Tachyon Size":0},"Emitted Task Size Warning":false}, - "Properties":{"France":"Paris","Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} + "greetings","Number of Tasks":200,"RDD Info":[],"Details":"details", + "Emitted Task Size Warning":false},"Properties":{"France":"Paris","Germany":"Berlin", + "Russia":"Moscow","Ukraine":"Kiev"}} """ private val stageCompletedJsonString = """ {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":101,"Stage Name": - "greetings","Number of Tasks":201,"RDD Info":{"RDD ID":101,"Name":"mayor","Storage + "greetings","Number of Tasks":201,"RDD Info":[{"RDD ID":101,"Name":"mayor","Storage Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":true, "Replication":1},"Number of Partitions":201,"Number of Cached Partitions":301, - "Memory Size":401,"Disk Size":501,"Tachyon Size":0},"Emitted Task Size Warning":false}} + "Memory Size":401,"Tachyon Size":0,"Disk Size":501}],"Details":"details", + "Emitted Task Size Warning":false}} """ private val taskStartJsonString = @@ -538,9 +537,9 @@ class JsonProtocolSuite extends FunSuite { 900,"Total Blocks Fetched":1500,"Remote Blocks Fetched":800,"Local Blocks Fetched": 700,"Fetch Wait Time":900,"Remote Bytes Read":1000},"Shuffle Write Metrics": {"Shuffle Bytes Written":1200,"Shuffle Write Time":1500},"Updated Blocks": - [{"Block ID":{"Type":"RDDBlockId","RDD ID":0,"Split Index":0},"Status": - {"Storage Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false, - "Deserialized":false,"Replication":2},"Memory Size":0,"Disk Size":0,"Tachyon Size":0}}]}} + [{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":true,"Use Memory":true, + "Use Tachyon":false,"Deserialized":false,"Replication":2},"Memory Size":0,"Tachyon Size":0, + "Disk Size":0}}]}} """ private val jobStartJsonString = From a162c9b337d99dd2a6102a80deb2a9707cdd93e9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 24 Jun 2014 19:04:29 -0700 Subject: [PATCH 034/124] [SPARK-2264][SQL] Fix failing CachedTableSuite Author: Michael Armbrust Closes #1201 from marmbrus/fixCacheTests and squashes the following commits: 9d87ed1 [Michael Armbrust] Use analyzer (which runs to fixed point) instead of manually removing analysis operators. --- .../org/apache/spark/sql/SQLContext.scala | 10 +++---- .../apache/spark/sql/CachedTableSuite.scala | 30 +++++++++++++++---- .../org/apache/spark/sql/SQLQuerySuite.scala | 19 ------------ 3 files changed, 29 insertions(+), 30 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 0bcfbf6f849e4..7195f9709d5e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -186,8 +186,8 @@ class SQLContext(@transient val sparkContext: SparkContext) /** Caches the specified table in-memory. */ def cacheTable(tableName: String): Unit = { - val currentTable = catalog.lookupRelation(None, tableName) - val asInMemoryRelation = EliminateAnalysisOperators(currentTable.logicalPlan) match { + val currentTable = table(tableName).queryExecution.analyzed + val asInMemoryRelation = currentTable match { case _: InMemoryRelation => currentTable.logicalPlan @@ -202,7 +202,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** Removes the specified table from the in-memory cache. */ def uncacheTable(tableName: String): Unit = { - EliminateAnalysisOperators(catalog.lookupRelation(None, tableName)) match { + table(tableName).queryExecution.analyzed match { // This is kind of a hack to make sure that if this was just an RDD registered as a table, // we reregister the RDD as a table. case inMem @ InMemoryRelation(_, _, e: ExistingRdd) => @@ -218,8 +218,8 @@ class SQLContext(@transient val sparkContext: SparkContext) /** Returns true if the table is currently cached in-memory. */ def isCached(tableName: String): Boolean = { - val relation = catalog.lookupRelation(None, tableName) - EliminateAnalysisOperators(relation) match { + val relation = table(tableName).queryExecution.analyzed + relation match { case _: InMemoryRelation => true case _ => false } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 128ddae11eb0d..c3c0dcb1aa00b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -20,13 +20,31 @@ package org.apache.spark.sql import org.apache.spark.sql.TestData._ import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.test.TestSQLContext._ class CachedTableSuite extends QueryTest { TestData // Load test tables. - // NOTE: ALL TESTS ARE IGNORED PENDING SPARK-2264 + test("SPARK-1669: cacheTable should be idempotent") { + assume(!table("testData").logicalPlan.isInstanceOf[InMemoryRelation]) - ignore("read from cached table and uncache") { + cacheTable("testData") + table("testData").queryExecution.analyzed match { + case _: InMemoryRelation => + case _ => + fail("testData should be cached") + } + + cacheTable("testData") + table("testData").queryExecution.analyzed match { + case InMemoryRelation(_, _, _: InMemoryColumnarTableScan) => + fail("cacheTable is not idempotent") + + case _ => + } + } + + test("read from cached table and uncache") { TestSQLContext.cacheTable("testData") checkAnswer( @@ -53,20 +71,20 @@ class CachedTableSuite extends QueryTest { } } - ignore("correct error on uncache of non-cached table") { + test("correct error on uncache of non-cached table") { intercept[IllegalArgumentException] { TestSQLContext.uncacheTable("testData") } } - ignore("SELECT Star Cached Table") { + test("SELECT Star Cached Table") { TestSQLContext.sql("SELECT * FROM testData").registerAsTable("selectStar") TestSQLContext.cacheTable("selectStar") TestSQLContext.sql("SELECT * FROM selectStar WHERE key = 1").collect() TestSQLContext.uncacheTable("selectStar") } - ignore("Self-join cached") { + test("Self-join cached") { val unCachedAnswer = TestSQLContext.sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key").collect() TestSQLContext.cacheTable("testData") @@ -76,7 +94,7 @@ class CachedTableSuite extends QueryTest { TestSQLContext.uncacheTable("testData") } - ignore("'CACHE TABLE' and 'UNCACHE TABLE' SQL statement") { + test("'CACHE TABLE' and 'UNCACHE TABLE' SQL statement") { TestSQLContext.sql("CACHE TABLE testData") TestSQLContext.table("testData").queryExecution.executedPlan match { case _: InMemoryColumnarTableScan => // Found evidence of caching 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 cca58c0063b38..bf7fafe952303 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 @@ -406,23 +406,4 @@ class SQLQuerySuite extends QueryTest { ) clear() } - - test("SPARK-1669: cacheTable should be idempotent") { - assume(!table("testData").logicalPlan.isInstanceOf[InMemoryRelation]) - - cacheTable("testData") - EliminateAnalysisOperators(table("testData").logicalPlan) match { - case _: InMemoryRelation => - case _ => - fail("testData should be cached") - } - - cacheTable("testData") - EliminateAnalysisOperators(table("testData").logicalPlan) match { - case InMemoryRelation(_, _, _: InMemoryColumnarTableScan) => - fail("cacheTable is not idempotent") - - case _ => - } - } } From 8ca41769fb16115a5a14ac842199d16cb28641ba Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 24 Jun 2014 19:06:07 -0700 Subject: [PATCH 035/124] [SPARK-1112, 2156] Bootstrap to fetch the driver's Spark properties. This is an alternative solution to #1124 . Before launching the executor backend, we first fetch driver's spark properties and use it to overwrite executor's spark properties. This should be better than #1124. @pwendell Are there spark properties that might be different on the driver and on the executors? Author: Xiangrui Meng Closes #1132 from mengxr/akka-bootstrap and squashes the following commits: 77ff32d [Xiangrui Meng] organize imports 68e1dfb [Xiangrui Meng] use timeout from AkkaUtils; remove props from RegisteredExecutor 46d332d [Xiangrui Meng] fix a test 7947c18 [Xiangrui Meng] increase slack size for akka 4ab696a [Xiangrui Meng] bootstrap to retrieve driver spark conf --- .../CoarseGrainedExecutorBackend.scala | 66 +++++++++++-------- .../org/apache/spark/executor/Executor.scala | 2 +- .../cluster/CoarseGrainedClusterMessage.scala | 6 +- .../CoarseGrainedSchedulerBackend.scala | 15 +++-- .../org/apache/spark/util/AkkaUtils.scala | 3 + .../CoarseGrainedSchedulerBackendSuite.scala | 2 +- 6 files changed, 54 insertions(+), 40 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 2279d77c91c89..b5fd334f40203 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -19,25 +19,26 @@ package org.apache.spark.executor import java.nio.ByteBuffer -import akka.actor._ -import akka.remote._ +import scala.concurrent.Await -import org.apache.spark.{SparkEnv, Logging, SecurityManager, SparkConf} +import akka.actor.{Actor, ActorSelection, Props} +import akka.pattern.Patterns +import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} + +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.scheduler.TaskDescription +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class CoarseGrainedExecutorBackend( driverUrl: String, executorId: String, hostPort: String, - cores: Int) - extends Actor - with ExecutorBackend - with Logging { + cores: Int, + sparkProperties: Seq[(String, String)]) extends Actor with ExecutorBackend with Logging { Utils.checkHostPort(hostPort, "Expected hostport") @@ -52,7 +53,7 @@ private[spark] class CoarseGrainedExecutorBackend( } override def receive = { - case RegisteredExecutor(sparkProperties) => + case RegisteredExecutor => logInfo("Successfully registered with driver") // Make this host instead of hostPort ? executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties, @@ -101,26 +102,33 @@ private[spark] object CoarseGrainedExecutorBackend { workerUrl: Option[String]) { SparkHadoopUtil.get.runAsSparkUser { () => - // Debug code - Utils.checkHost(hostname) - - val conf = new SparkConf - // Create a new ActorSystem to run the backend, because we can't create a - // SparkEnv / Executor before getting started with all our system properties, etc - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, - conf, new SecurityManager(conf)) - // set it - val sparkHostPort = hostname + ":" + boundPort - actorSystem.actorOf( - Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, - sparkHostPort, cores), - name = "Executor") - workerUrl.foreach { - url => - actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") - } - actorSystem.awaitTermination() - + // Debug code + Utils.checkHost(hostname) + + // Bootstrap to fetch the driver's Spark properties. + val executorConf = new SparkConf + val (fetcher, _) = AkkaUtils.createActorSystem( + "driverPropsFetcher", hostname, 0, executorConf, new SecurityManager(executorConf)) + val driver = fetcher.actorSelection(driverUrl) + val timeout = AkkaUtils.askTimeout(executorConf) + val fut = Patterns.ask(driver, RetrieveSparkProps, timeout) + val props = Await.result(fut, timeout).asInstanceOf[Seq[(String, String)]] + fetcher.shutdown() + + // Create a new ActorSystem using driver's Spark properties to run the backend. + val driverConf = new SparkConf().setAll(props) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + "sparkExecutor", hostname, 0, driverConf, new SecurityManager(driverConf)) + // set it + val sparkHostPort = hostname + ":" + boundPort + actorSystem.actorOf( + Props(classOf[CoarseGrainedExecutorBackend], + driverUrl, executorId, sparkHostPort, cores, props), + name = "Executor") + workerUrl.foreach { url => + actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") + } + actorSystem.awaitTermination() } } 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 baee7a216a7c3..557b9a3f46a08 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -212,7 +212,7 @@ private[spark] class Executor( val serializedDirectResult = ser.serialize(directResult) logInfo("Serialized size of result for " + taskId + " is " + serializedDirectResult.limit) val serializedResult = { - if (serializedDirectResult.limit >= akkaFrameSize - 1024) { + if (serializedDirectResult.limit >= akkaFrameSize - AkkaUtils.reservedSizeBytes) { logInfo("Storing result for " + taskId + " in local BlockManager") val blockId = TaskResultBlockId(taskId) env.blockManager.putBytes( diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index ca74069ef885c..318e16552201c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -20,21 +20,21 @@ package org.apache.spark.scheduler.cluster import java.nio.ByteBuffer import org.apache.spark.TaskState.TaskState -import org.apache.spark.scheduler.TaskDescription import org.apache.spark.util.{SerializableBuffer, Utils} private[spark] sealed trait CoarseGrainedClusterMessage extends Serializable private[spark] object CoarseGrainedClusterMessages { + case object RetrieveSparkProps extends CoarseGrainedClusterMessage + // Driver to executors case class LaunchTask(data: SerializableBuffer) extends CoarseGrainedClusterMessage case class KillTask(taskId: Long, executor: String, interruptThread: Boolean) extends CoarseGrainedClusterMessage - case class RegisteredExecutor(sparkProperties: Seq[(String, String)]) - extends CoarseGrainedClusterMessage + case object RegisteredExecutor extends CoarseGrainedClusterMessage case class RegisterExecutorFailed(message: String) extends CoarseGrainedClusterMessage diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index e47a060683a2d..05d01b0c821f9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -75,7 +75,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A sender ! RegisterExecutorFailed("Duplicate executor ID: " + executorId) } else { logInfo("Registered executor: " + sender + " with ID " + executorId) - sender ! RegisteredExecutor(sparkProperties) + sender ! RegisteredExecutor executorActor(executorId) = sender executorHost(executorId) = Utils.parseHostPort(hostPort)._1 totalCores(executorId) = cores @@ -124,6 +124,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated")) + case RetrieveSparkProps => + sender ! sparkProperties } // Make fake resource offers on all executors @@ -143,14 +145,15 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A for (task <- tasks.flatten) { val ser = SparkEnv.get.closureSerializer.newInstance() val serializedTask = ser.serialize(task) - if (serializedTask.limit >= akkaFrameSize - 1024) { + if (serializedTask.limit >= akkaFrameSize - AkkaUtils.reservedSizeBytes) { val taskSetId = scheduler.taskIdToTaskSetId(task.taskId) scheduler.activeTaskSets.get(taskSetId).foreach { taskSet => try { - var msg = "Serialized task %s:%d was %d bytes which " + - "exceeds spark.akka.frameSize (%d bytes). " + - "Consider using broadcast variables for large values." - msg = msg.format(task.taskId, task.index, serializedTask.limit, akkaFrameSize) + var msg = "Serialized task %s:%d was %d bytes, which exceeds max allowed: " + + "spark.akka.frameSize (%d bytes) - reserved (%d bytes). Consider increasing " + + "spark.akka.frameSize or using broadcast variables for large values." + msg = msg.format(task.taskId, task.index, serializedTask.limit, akkaFrameSize, + AkkaUtils.reservedSizeBytes) taskSet.abort(msg) } catch { case e: Exception => logError("Exception in error callback", e) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index a8d12bb2a0165..9930c717492f2 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -121,4 +121,7 @@ private[spark] object AkkaUtils extends Logging { def maxFrameSizeBytes(conf: SparkConf): Int = { conf.getInt("spark.akka.frameSize", 10) * 1024 * 1024 } + + /** Space reserved for extra data in an Akka message besides serialized task or task result. */ + val reservedSizeBytes = 200 * 1024 } diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index efef9d26dadca..f77661ccbd1c5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -35,7 +35,7 @@ class CoarseGrainedSchedulerBackendSuite extends FunSuite with LocalSparkContext val thrown = intercept[SparkException] { larger.collect() } - assert(thrown.getMessage.contains("Consider using broadcast variables for large values")) + assert(thrown.getMessage.contains("using broadcast variables for large values")) val smaller = sc.parallelize(1 to 4).collect() assert(smaller.size === 4) } From 133495d82672c3f34d40a6298cc80c31f91faf5c Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 24 Jun 2014 19:07:02 -0700 Subject: [PATCH 036/124] [SQL]Add base row updating methods for JoinedRow This will be helpful in join operators. Author: Cheng Hao Closes #1187 from chenghao-intel/joinedRow and squashes the following commits: 87c19e3 [Cheng Hao] Add base row set methods for JoinedRow --- .../sql/catalyst/expressions/Projection.scala | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index c9b7cea6a3e5f..a9e976c3abaf2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -81,6 +81,18 @@ class JoinedRow extends Row { this } + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + def iterator = row1.iterator ++ row2.iterator def length = row1.length + row2.length @@ -124,4 +136,9 @@ class JoinedRow extends Row { } new GenericRow(copiedValues) } + + override def toString() = { + val row = (if (row1 != null) row1 else Seq[Any]()) ++ (if (row2 != null) row2 else Seq[Any]()) + s"[${row.mkString(",")}]" + } } From 54055fb2b73a545e7cca6c645356ef287aa5a7c4 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Tue, 24 Jun 2014 19:31:20 -0700 Subject: [PATCH 037/124] Autodetect JAVA_HOME on RPM-based systems Author: Matthew Farrellee Closes #1185 from mattf/master-1 and squashes the following commits: 42150fc [Matthew Farrellee] Autodetect JAVA_HOME on RPM-based systems --- make-distribution.sh | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/make-distribution.sh b/make-distribution.sh index 6f708e0ebff6c..86868438e75c3 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -84,6 +84,17 @@ while (( "$#" )); do shift done +if [ -z "$JAVA_HOME" ]; then + # Fall back on JAVA_HOME from rpm, if found + if which rpm &>/dev/null; then + RPM_JAVA_HOME=$(rpm -E %java_home 2>/dev/null) + if [ "$RPM_JAVA_HOME" != "%java_home" ]; then + JAVA_HOME=$RPM_JAVA_HOME + echo "No JAVA_HOME set, proceeding with '$JAVA_HOME' learned from rpm" + fi + fi +fi + if [ -z "$JAVA_HOME" ]; then echo "Error: JAVA_HOME is not set, cannot proceed." exit -1 From 2714968e1b40221739c5dfba7ca4c0c06953dbe2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 24 Jun 2014 19:39:19 -0700 Subject: [PATCH 038/124] Fix possible null pointer in acumulator toString Author: Michael Armbrust Closes #1204 from marmbrus/nullPointerToString and squashes the following commits: 35b5fce [Michael Armbrust] Fix possible null pointer in acumulator toString --- core/src/main/scala/org/apache/spark/Accumulators.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index cdfd338081fa2..9c55bfbb47626 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -127,7 +127,7 @@ class Accumulable[R, T] ( Accumulators.register(this, false) } - override def toString = value_.toString + override def toString = if (value_ == null) "null" else value_.toString } /** From b6b44853cd61660f2917b99d87c157e2b4430e5c Mon Sep 17 00:00:00 2001 From: witgo Date: Tue, 24 Jun 2014 19:44:37 -0700 Subject: [PATCH 039/124] SPARK-2248: spark.default.parallelism does not apply in local mode Author: witgo Closes #1194 from witgo/SPARK-2248 and squashes the following commits: 6ac950b [witgo] spark.default.parallelism does not apply in local mode --- .../spark/scheduler/local/LocalBackend.scala | 3 ++- .../SparkContextSchedulerCreationSuite.scala | 16 ++++++++++++++++ 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 43f0e18a0cbe0..9b95ccca0443e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -97,7 +97,8 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: localActor ! ReviveOffers } - override def defaultParallelism() = totalCores + override def defaultParallelism() = + scheduler.conf.getInt("spark.default.parallelism", totalCores) override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { localActor ! KillTask(taskId, interruptThread) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 94fba102865b3..67e3be21c3c93 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -77,6 +77,22 @@ class SparkContextSchedulerCreationSuite } } + test("local-default-parallelism") { + val defaultParallelism = System.getProperty("spark.default.parallelism") + System.setProperty("spark.default.parallelism", "16") + val sched = createTaskScheduler("local") + + sched.backend match { + case s: LocalBackend => assert(s.defaultParallelism() === 16) + case _ => fail() + } + + Option(defaultParallelism) match { + case Some(v) => System.setProperty("spark.default.parallelism", v) + case _ => System.clearProperty("spark.default.parallelism") + } + } + test("simr") { createTaskScheduler("simr://uri").backend match { case s: SimrSchedulerBackend => // OK From 8fade8973e5fc97f781de5344beb66b90bd6e524 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 25 Jun 2014 00:14:34 -0700 Subject: [PATCH 040/124] [SPARK-2263][SQL] Support inserting MAP to Hive tables JIRA issue: [SPARK-2263](https://issues.apache.org/jira/browse/SPARK-2263) Map objects were not converted to Hive types before inserting into Hive tables. Author: Cheng Lian Closes #1205 from liancheng/spark-2263 and squashes the following commits: c7a4373 [Cheng Lian] Addressed @concretevitamin's comment 784940b [Cheng Lian] SARPK-2263: support inserting MAP to Hive tables --- .../sql/hive/execution/InsertIntoHiveTable.scala | 8 ++++++++ .../spark/sql/hive/execution/HiveQuerySuite.scala | 15 ++++++++++++--- .../spark/sql/hive/execution/HiveUdfSuite.scala | 3 --- 3 files changed, 20 insertions(+), 6 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 594a803806ede..c2b0b00aa5852 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.hive.execution import scala.collection.JavaConversions._ +import java.util.{HashMap => JHashMap} + import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} import org.apache.hadoop.hive.metastore.MetaStoreUtils import org.apache.hadoop.hive.ql.Context @@ -88,6 +90,12 @@ case class InsertIntoHiveTable( val wrappedSeq = s.map(wrap(_, oi.getListElementObjectInspector)) seqAsJavaList(wrappedSeq) + case (m: Map[_, _], oi: MapObjectInspector) => + val keyOi = oi.getMapKeyObjectInspector + val valueOi = oi.getMapValueObjectInspector + val wrappedMap = m.map { case (key, value) => wrap(key, keyOi) -> wrap(value, valueOi) } + mapAsJavaMap(wrappedMap) + case (obj, _) => obj } 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 d855310253bf3..9f1cd703103ec 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 @@ -228,7 +228,7 @@ class HiveQuerySuite extends HiveComparisonTest { val fixture = List(("foo", 2), ("bar", 1), ("foo", 4), ("bar", 3)) .zipWithIndex.map {case Pair(Pair(value, attr), key) => HavingRow(key, value, attr)} TestHive.sparkContext.parallelize(fixture).registerAsTable("having_test") - val results = + val results = hql("SELECT value, max(attr) AS attr FROM having_test GROUP BY value HAVING attr > 3") .collect() .map(x => Pair(x.getString(0), x.getInt(1))) @@ -236,7 +236,7 @@ class HiveQuerySuite extends HiveComparisonTest { assert(results === Array(Pair("foo", 4))) TestHive.reset() } - + test("SPARK-2180: HAVING with non-boolean clause raises no exceptions") { hql("select key, count(*) c from src group by key having c").collect() } @@ -370,6 +370,16 @@ class HiveQuerySuite extends HiveComparisonTest { } } + test("SPARK-2263: Insert Map values") { + hql("CREATE TABLE m(value MAP)") + hql("INSERT OVERWRITE TABLE m SELECT MAP(key, value) FROM src LIMIT 10") + hql("SELECT * FROM m").collect().zip(hql("SELECT * FROM src LIMIT 10").collect()).map { + case (Row(map: Map[Int, String]), Row(key: Int, value: String)) => + assert(map.size === 1) + assert(map.head === (key, value)) + } + } + test("parse HQL set commands") { // Adapted from its SQL counterpart. val testKey = "spark.sql.key.usedfortestonly" @@ -460,7 +470,6 @@ class HiveQuerySuite extends HiveComparisonTest { // Put tests that depend on specific Hive settings before these last two test, // since they modify /clear stuff. - } // for SPARK-2180 test diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index a9e3f42a3adfc..f944d010660eb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -122,6 +122,3 @@ class PairUdf extends GenericUDF { override def getDisplayString(p1: Array[String]): String = "" } - - - From 22036aeb1b2cac7f48cd60afea925b42a5318631 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 25 Jun 2014 00:17:28 -0700 Subject: [PATCH 041/124] [BUGFIX][SQL] Should match java.math.BigDecimal when wnrapping Hive output The `BigDecimal` branch in `unwrap` matches to `scala.math.BigDecimal` rather than `java.math.BigDecimal`. Author: Cheng Lian Closes #1199 from liancheng/javaBigDecimal and squashes the following commits: e9bb481 [Cheng Lian] Should match java.math.BigDecimal when wnrapping Hive output --- .../main/scala/org/apache/spark/sql/hive/hiveUdfs.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index ad5e24c62c621..9b105308ab7cf 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -84,7 +84,7 @@ private[hive] object HiveFunctionRegistry case c: Class[_] if c == classOf[hadoopIo.FloatWritable] => FloatType case c: Class[_] if c == classOf[hadoopIo.BooleanWritable] => BooleanType case c: Class[_] if c == classOf[hadoopIo.BytesWritable] => BinaryType - + // java class case c: Class[_] if c == classOf[java.lang.String] => StringType case c: Class[_] if c == classOf[java.sql.Timestamp] => TimestampType @@ -98,7 +98,7 @@ private[hive] object HiveFunctionRegistry case c: Class[_] if c == classOf[java.lang.Byte] => ByteType case c: Class[_] if c == classOf[java.lang.Float] => FloatType case c: Class[_] if c == classOf[java.lang.Boolean] => BooleanType - + // primitive type case c: Class[_] if c == java.lang.Short.TYPE => ShortType case c: Class[_] if c == java.lang.Integer.TYPE => IntegerType @@ -107,7 +107,7 @@ private[hive] object HiveFunctionRegistry case c: Class[_] if c == java.lang.Byte.TYPE => ByteType case c: Class[_] if c == java.lang.Float.TYPE => FloatType case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType - + case c: Class[_] if c.isArray => ArrayType(javaClassToDataType(c.getComponentType)) } } @@ -148,7 +148,7 @@ private[hive] trait HiveFunctionFactory { case p: java.lang.Byte => p case p: java.lang.Boolean => p case str: String => str - case p: BigDecimal => p + case p: java.math.BigDecimal => p case p: Array[Byte] => p case p: java.sql.Timestamp => p } From acc01ab3265c317f36a4fca28d3b9d72b0096c12 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Wed, 25 Jun 2014 00:23:32 -0700 Subject: [PATCH 042/124] SPARK-2038: rename "conf" parameters in the saveAsHadoop functions with source-compatibility https://issues.apache.org/jira/browse/SPARK-2038 to differentiate with SparkConf object and at the same time keep the source level compatibility Author: CodingCat Closes #1137 from CodingCat/SPARK-2038 and squashes the following commits: 11abeba [CodingCat] revise the comments 7ee5712 [CodingCat] to keep the source-compatibility 763975f [CodingCat] style fix d91288d [CodingCat] rename "conf" parameters in the saveAsHadoop functions --- .../apache/spark/rdd/PairRDDFunctions.scala | 49 +++++++++++-------- 1 file changed, 29 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 2b2b9ae3fd533..fc9beb166befe 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -762,7 +762,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) outputFormatClass: Class[_ <: NewOutputFormat[_, _]], conf: Configuration = self.context.hadoopConfiguration) { - val job = new NewAPIHadoopJob(conf) + // Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038). + val hadoopConf = conf + val job = new NewAPIHadoopJob(hadoopConf) job.setOutputKeyClass(keyClass) job.setOutputValueClass(valueClass) job.setOutputFormatClass(outputFormatClass) @@ -795,22 +797,25 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) outputFormatClass: Class[_ <: OutputFormat[_, _]], conf: JobConf = new JobConf(self.context.hadoopConfiguration), codec: Option[Class[_ <: CompressionCodec]] = None) { - conf.setOutputKeyClass(keyClass) - conf.setOutputValueClass(valueClass) + // Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038). + val hadoopConf = conf + hadoopConf.setOutputKeyClass(keyClass) + hadoopConf.setOutputValueClass(valueClass) // Doesn't work in Scala 2.9 due to what may be a generics bug // TODO: Should we uncomment this for Scala 2.10? // conf.setOutputFormat(outputFormatClass) - conf.set("mapred.output.format.class", outputFormatClass.getName) + hadoopConf.set("mapred.output.format.class", outputFormatClass.getName) for (c <- codec) { - conf.setCompressMapOutput(true) - conf.set("mapred.output.compress", "true") - conf.setMapOutputCompressorClass(c) - conf.set("mapred.output.compression.codec", c.getCanonicalName) - conf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) + hadoopConf.setCompressMapOutput(true) + hadoopConf.set("mapred.output.compress", "true") + hadoopConf.setMapOutputCompressorClass(c) + hadoopConf.set("mapred.output.compression.codec", c.getCanonicalName) + hadoopConf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) } - conf.setOutputCommitter(classOf[FileOutputCommitter]) - FileOutputFormat.setOutputPath(conf, SparkHadoopWriter.createPathFromString(path, conf)) - saveAsHadoopDataset(conf) + hadoopConf.setOutputCommitter(classOf[FileOutputCommitter]) + FileOutputFormat.setOutputPath(hadoopConf, + SparkHadoopWriter.createPathFromString(path, hadoopConf)) + saveAsHadoopDataset(hadoopConf) } /** @@ -820,7 +825,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * configured for a Hadoop MapReduce job. */ def saveAsNewAPIHadoopDataset(conf: Configuration) { - val job = new NewAPIHadoopJob(conf) + // Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038). + val hadoopConf = conf + val job = new NewAPIHadoopJob(hadoopConf) val formatter = new SimpleDateFormat("yyyyMMddHHmm") val jobtrackerID = formatter.format(new Date()) val stageId = self.id @@ -877,9 +884,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * MapReduce job. */ def saveAsHadoopDataset(conf: JobConf) { - val outputFormatInstance = conf.getOutputFormat - val keyClass = conf.getOutputKeyClass - val valueClass = conf.getOutputValueClass + // Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038). + val hadoopConf = conf + val outputFormatInstance = hadoopConf.getOutputFormat + val keyClass = hadoopConf.getOutputKeyClass + val valueClass = hadoopConf.getOutputValueClass if (outputFormatInstance == null) { throw new SparkException("Output format class not set") } @@ -889,18 +898,18 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) if (valueClass == null) { throw new SparkException("Output value class not set") } - SparkHadoopUtil.get.addCredentials(conf) + SparkHadoopUtil.get.addCredentials(hadoopConf) logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + valueClass.getSimpleName + ")") if (self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true)) { // FileOutputFormat ignores the filesystem parameter - val ignoredFs = FileSystem.get(conf) - conf.getOutputFormat.checkOutputSpecs(ignoredFs, conf) + val ignoredFs = FileSystem.get(hadoopConf) + hadoopConf.getOutputFormat.checkOutputSpecs(ignoredFs, hadoopConf) } - val writer = new SparkHadoopWriter(conf) + val writer = new SparkHadoopWriter(hadoopConf) writer.preSetup() def writeToFile(context: TaskContext, iter: Iterator[(K, V)]) { From ac06a85da59db8f2654cdf6601d186348da09c01 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 25 Jun 2014 01:01:23 -0700 Subject: [PATCH 043/124] Replace doc reference to Shark with Spark SQL. --- docs/index.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/index.md b/docs/index.md index 1a4ff3dbf57be..4ac0982ae54f1 100644 --- a/docs/index.md +++ b/docs/index.md @@ -6,7 +6,7 @@ title: Spark Overview Apache Spark is a fast and general-purpose cluster computing system. It provides high-level APIs in Java, Scala and Python, and an optimized engine that supports general execution graphs. -It also supports a rich set of higher-level tools including [Shark](http://shark.cs.berkeley.edu) (Hive on Spark), [Spark SQL](sql-programming-guide.html) for structured data, [MLlib](mllib-guide.html) for machine learning, [GraphX](graphx-programming-guide.html) for graph processing, and [Spark Streaming](streaming-programming-guide.html). +It also supports a rich set of higher-level tools including [Spark SQL](sql-programming-guide.html) for SQL and structured data processing, [MLlib](mllib-guide.html) for machine learning, [GraphX](graphx-programming-guide.html) for graph processing, and [Spark Streaming](streaming-programming-guide.html). # Downloading @@ -109,10 +109,9 @@ options for deployment: **External Resources:** * [Spark Homepage](http://spark.apache.org) -* [Shark](http://shark.cs.berkeley.edu): Apache Hive over Spark * [Mailing Lists](http://spark.apache.org/mailing-lists.html): ask questions about Spark here * [AMP Camps](http://ampcamp.berkeley.edu/): a series of training camps at UC Berkeley that featured talks and - exercises about Spark, Shark, Spark Streaming, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/3/), + exercises about Spark, Spark Streaming, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/3/), [slides](http://ampcamp.berkeley.edu/3/) and [exercises](http://ampcamp.berkeley.edu/3/exercises/) are available online for free. * [Code Examples](http://spark.apache.org/examples.html): more are also available in the `examples` subfolder of Spark ([Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples), From 5603e4c47f1dc1b87336f57ed4d6bd9e88f5abcc Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 25 Jun 2014 10:47:22 -0700 Subject: [PATCH 044/124] [SPARK-2242] HOTFIX: pyspark shell hangs on simple job This reverts a change introduced in 3870248740d83b0292ccca88a494ce19783847f0, which redirected all stderr to the OS pipe instead of directly to the `bin/pyspark` shell output. This causes a simple job to hang in two ways: 1. If the cluster is not configured correctly or does not have enough resources, the job hangs without producing any output, because the relevant warning messages are masked. 2. If the stderr volume is large, this could lead to a deadlock if we redirect everything to the OS pipe. From the [python docs](https://docs.python.org/2/library/subprocess.html): ``` Note Do not use stdout=PIPE or stderr=PIPE with this function as that can deadlock based on the child process output volume. Use Popen with the communicate() method when you need pipes. ``` Note that we cannot remove `stdout=PIPE` in a similar way, because we currently use it to communicate the py4j port. However, it should be fine (as it has been for a long time) because we do not produce a ton of traffic through `stdout`. That commit was not merged in branch-1.0, so this fix is for master only. Author: Andrew Or Closes #1178 from andrewor14/fix-python and squashes the following commits: e68e870 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-python 20849a8 [Andrew Or] Tone down stdout interference message a09805b [Andrew Or] Return more than 1 line of error message to user 6dfbd1e [Andrew Or] Don't swallow original exception 0d1861f [Andrew Or] Provide more helpful output if stdout is garbled 21c9d7c [Andrew Or] Do not mask stderr from output --- python/pyspark/java_gateway.py | 21 +++++++++++++-------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 19235d5f79f85..0dbead4415b02 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -43,18 +43,23 @@ def launch_gateway(): # Don't send ctrl-c / SIGINT to the Java gateway: def preexec_func(): signal.signal(signal.SIGINT, signal.SIG_IGN) - proc = Popen(command, stdout=PIPE, stdin=PIPE, stderr=PIPE, preexec_fn=preexec_func) + proc = Popen(command, stdout=PIPE, stdin=PIPE, preexec_fn=preexec_func) else: # preexec_fn not supported on Windows - proc = Popen(command, stdout=PIPE, stdin=PIPE, stderr=PIPE) - + proc = Popen(command, stdout=PIPE, stdin=PIPE) + try: # Determine which ephemeral port the server started on: - gateway_port = int(proc.stdout.readline()) - except: - error_code = proc.poll() - raise Exception("Launching GatewayServer failed with exit code %d: %s" % - (error_code, "".join(proc.stderr.readlines()))) + gateway_port = proc.stdout.readline() + gateway_port = int(gateway_port) + except ValueError: + (stdout, _) = proc.communicate() + exit_code = proc.poll() + error_msg = "Launching GatewayServer failed" + error_msg += " with exit code %d!" % exit_code if exit_code else "! " + error_msg += "(Warning: unexpected output detected.)\n\n" + error_msg += gateway_port + stdout + raise Exception(error_msg) # Create a thread to echo output from the GatewayServer, which is required # for Java log output to show up: From 9aa603296c285e1acf4bde64583f203008ba3e91 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 25 Jun 2014 12:23:08 -0700 Subject: [PATCH 045/124] [SPARK-2258 / 2266] Fix a few worker UI bugs **SPARK-2258.** Worker UI displays zombie processes if the executor throws an exception before a process is launched. This is because we only inform the Worker of the change if the process is already launched, which in this case it isn't. **SPARK-2266.** We expose "Some(app-id)" on the log page. This is fairly minor. Author: Andrew Or Closes #1213 from andrewor14/fix-worker-ui and squashes the following commits: c1223fe [Andrew Or] Fix worker UI bugs --- .../org/apache/spark/deploy/worker/ExecutorRunner.scala | 5 +++-- .../scala/org/apache/spark/deploy/worker/ui/LogPage.scala | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 6433aac1c23e0..467317dd9b44c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -77,6 +77,7 @@ private[spark] class ExecutorRunner( * @param message the exception message which caused the executor's death */ private def killProcess(message: Option[String]) { + var exitCode: Option[Int] = None if (process != null) { logInfo("Killing process!") process.destroy() @@ -87,9 +88,9 @@ private[spark] class ExecutorRunner( if (stderrAppender != null) { stderrAppender.stop() } - val exitCode = process.waitFor() - worker ! ExecutorStateChanged(appId, execId, state, message, Some(exitCode)) + exitCode = Some(process.waitFor()) } + worker ! ExecutorStateChanged(appId, execId, state, message, exitCode) } /** Stop this executor runner, including killing the process it launched */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index 6a5ffb1b71bfb..b389cb546de6c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -120,7 +120,7 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") w - UIUtils.basicSparkPage(content, logType + " log page for " + appId) + UIUtils.basicSparkPage(content, logType + " log page for " + appId.getOrElse("unknown app")) } /** Get the part of the log files given the offset and desired length of bytes */ From 7ff2c754f340ba4c4077b0ff6285876eb7871c7b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 25 Jun 2014 12:43:22 -0700 Subject: [PATCH 046/124] [SPARK-2270] Kryo cannot serialize results returned by asJavaIterable and thus groupBy/cogroup are broken in Java APIs when Kryo is used). @pwendell this should be merged into 1.0.1. Thanks @sorenmacbeth for reporting this & helping out with the fix. Author: Reynold Xin Closes #1206 from rxin/kryo-iterable-2270 and squashes the following commits: 09da0aa [Reynold Xin] Updated the comment. 009bf64 [Reynold Xin] [SPARK-2270] Kryo cannot serialize results returned by asJavaIterable (and thus groupBy/cogroup are broken in Java APIs when Kryo is used). --- .../spark/serializer/KryoSerializer.scala | 50 +++++++++++++++++++ .../serializer/KryoSerializerSuite.scala | 15 ++++++ 2 files changed, 65 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 5286f7b4c211a..82b62aaf61521 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -64,6 +64,9 @@ class KryoSerializer(conf: SparkConf) kryo.register(cls) } + // For results returned by asJavaIterable. See JavaIterableWrapperSerializer. + kryo.register(JavaIterableWrapperSerializer.wrapperClass, new JavaIterableWrapperSerializer) + // Allow sending SerializableWritable kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) @@ -183,3 +186,50 @@ private[serializer] object KryoSerializer { classOf[Array[Byte]] ) } + +/** + * A Kryo serializer for serializing results returned by asJavaIterable. + * + * The underlying object is scala.collection.convert.Wrappers$IterableWrapper. + * Kryo deserializes this into an AbstractCollection, which unfortunately doesn't work. + */ +private class JavaIterableWrapperSerializer + extends com.esotericsoftware.kryo.Serializer[java.lang.Iterable[_]] { + + import JavaIterableWrapperSerializer._ + + override def write(kryo: Kryo, out: KryoOutput, obj: java.lang.Iterable[_]): Unit = { + // If the object is the wrapper, simply serialize the underlying Scala Iterable object. + // Otherwise, serialize the object itself. + if (obj.getClass == wrapperClass && underlyingMethodOpt.isDefined) { + kryo.writeClassAndObject(out, underlyingMethodOpt.get.invoke(obj)) + } else { + kryo.writeClassAndObject(out, obj) + } + } + + override def read(kryo: Kryo, in: KryoInput, clz: Class[java.lang.Iterable[_]]) + : java.lang.Iterable[_] = { + kryo.readClassAndObject(in) match { + case scalaIterable: Iterable[_] => + scala.collection.JavaConversions.asJavaIterable(scalaIterable) + case javaIterable: java.lang.Iterable[_] => + javaIterable + } + } +} + +private object JavaIterableWrapperSerializer extends Logging { + // The class returned by asJavaIterable (scala.collection.convert.Wrappers$IterableWrapper). + val wrapperClass = + scala.collection.convert.WrapAsJava.asJavaIterable(Seq(1)).getClass + + // Get the underlying method so we can use it to get the Scala collection for serialization. + private val underlyingMethodOpt = { + try Some(wrapperClass.getDeclaredMethod("underlying")) catch { + case e: Exception => + logError("Failed to find the underlying field in " + wrapperClass, e) + None + } + } +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index cdd6b3d8feed7..79280d1a06653 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -128,6 +128,21 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { check(1.0 until 1000000.0 by 2.0) } + test("asJavaIterable") { + // Serialize a collection wrapped by asJavaIterable + val ser = new KryoSerializer(conf).newInstance() + val a = ser.serialize(scala.collection.convert.WrapAsJava.asJavaIterable(Seq(12345))) + val b = ser.deserialize[java.lang.Iterable[Int]](a) + assert(b.iterator().next() === 12345) + + // Serialize a normal Java collection + val col = new java.util.ArrayList[Int] + col.add(54321) + val c = ser.serialize(col) + val d = ser.deserialize[java.lang.Iterable[Int]](c) + assert(b.iterator().next() === 12345) + } + test("custom registrator") { val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { From 1132e472eca1a00c2ce10d2f84e8f0e79a5193d3 Mon Sep 17 00:00:00 2001 From: Sebastien Rainville Date: Wed, 25 Jun 2014 13:21:18 -0700 Subject: [PATCH 047/124] [SPARK-2204] Launch tasks on the proper executors in mesos fine-grained mode The scheduler for Mesos in fine-grained mode launches tasks on the wrong executors. `MesosSchedulerBackend.resourceOffers(SchedulerDriver, List[Offer])` is assuming that `TaskSchedulerImpl.resourceOffers(Seq[WorkerOffer])` is returning task lists in the same order as the offers it was passed, but in the current implementation `TaskSchedulerImpl.resourceOffers` shuffles the offers to avoid assigning the tasks always to the same executors. The result is that the tasks are launched on the wrong executors. The jobs are sometimes able to complete, but most of the time they fail. It seems that as soon as something goes wrong with a task for some reason Spark is not able to recover since it's mistaken as to where the tasks are actually running. Also, it seems that the more the cluster is under load the more likely the job is to fail because there's a higher probability that Spark is trying to launch a task on a slave that doesn't actually have enough resources, again because it's using the wrong offers. The solution is to not assume that the order in which the tasks are returned is the same as the offers, and simply launch the tasks on the executor decided by `TaskSchedulerImpl.resourceOffers`. What I am not sure about is that I considered slaveId and executorId to be the same, which is true at least in my setup, but I don't know if that is always true. I tested this on top of the 1.0.0 release and it seems to work fine on our cluster. Author: Sebastien Rainville Closes #1140 from sebastienrainville/fine-grained-mode-fix-master and squashes the following commits: a98b0e0 [Sebastien Rainville] Use a HashMap to retrieve the offer indices d6ffe54 [Sebastien Rainville] Launch tasks on the proper executors in mesos fine-grained mode --- .../cluster/mesos/MesosSchedulerBackend.scala | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index a089a02d42170..c717e7c621a8f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -185,8 +185,8 @@ private[spark] class MesosSchedulerBackend( synchronized { // Build a big list of the offerable workers, and remember their indices so that we can // figure out which Offer to reply to for each worker - val offerableIndices = new ArrayBuffer[Int] val offerableWorkers = new ArrayBuffer[WorkerOffer] + val offerableIndices = new HashMap[String, Int] def enoughMemory(o: Offer) = { val mem = getResource(o.getResourcesList, "mem") @@ -195,7 +195,7 @@ private[spark] class MesosSchedulerBackend( } for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) { - offerableIndices += index + offerableIndices.put(offer.getSlaveId.getValue, index) offerableWorkers += new WorkerOffer( offer.getSlaveId.getValue, offer.getHostname, @@ -206,14 +206,13 @@ private[spark] class MesosSchedulerBackend( val taskLists = scheduler.resourceOffers(offerableWorkers) // Build a list of Mesos tasks for each slave - val mesosTasks = offers.map(o => Collections.emptyList[MesosTaskInfo]()) + val mesosTasks = offers.map(o => new JArrayList[MesosTaskInfo]()) for ((taskList, index) <- taskLists.zipWithIndex) { if (!taskList.isEmpty) { - val offerNum = offerableIndices(index) - val slaveId = offers(offerNum).getSlaveId.getValue - slaveIdsWithExecutors += slaveId - mesosTasks(offerNum) = new JArrayList[MesosTaskInfo](taskList.size) for (taskDesc <- taskList) { + val slaveId = taskDesc.executorId + val offerNum = offerableIndices(slaveId) + slaveIdsWithExecutors += slaveId taskIdToSlaveId(taskDesc.taskId) = slaveId mesosTasks(offerNum).add(createMesosTask(taskDesc, slaveId)) } From 9d824fed8c62dd6c87b4c855c2fea930c01b58f4 Mon Sep 17 00:00:00 2001 From: Zongheng Yang Date: Wed, 25 Jun 2014 18:06:33 -0700 Subject: [PATCH 048/124] [SQL] SPARK-1800 Add broadcast hash join operator & associated hints. This PR is based off Michael's [PR 734](https://github.com/apache/spark/pull/734) and includes a bunch of cleanups. Moreover, this PR also - makes `SparkLogicalPlan` take a `tableName: String`, which facilitates testing. - moves join-related tests to a single file. Author: Zongheng Yang Author: Michael Armbrust Closes #1163 from concretevitamin/auto-broadcast-hash-join and squashes the following commits: d0f4991 [Zongheng Yang] Fix bug in broadcast hash join & add test to cover it. af080d7 [Zongheng Yang] Fix in joinIterators()'s next(). 440d277 [Zongheng Yang] Fixes to imports; add back requiredChildDistribution (lost when merging) 208d5f6 [Zongheng Yang] Make LeftSemiJoinHash mix in HashJoin. ad6c7cc [Zongheng Yang] Minor cleanups. 814b3bf [Zongheng Yang] Merge branch 'master' into auto-broadcast-hash-join a8a093e [Zongheng Yang] Minor cleanups. 6fd8443 [Zongheng Yang] Cut down size estimation related stuff. a4267be [Zongheng Yang] Add test for broadcast hash join and related necessary refactorings: 0e64b08 [Zongheng Yang] Scalastyle fix. 91461c2 [Zongheng Yang] Merge branch 'master' into auto-broadcast-hash-join 7c7158b [Zongheng Yang] Prototype of auto conversion to broadcast hash join. 0ad122f [Zongheng Yang] Merge branch 'master' into auto-broadcast-hash-join 3e5d77c [Zongheng Yang] WIP: giant and messy WIP. a92ed0c [Michael Armbrust] Formatting. 76ca434 [Michael Armbrust] A simple strategy that broadcasts tables only when they are found in a configuration hint. cf6b381 [Michael Armbrust] Split out generic logic for hash joins and create two concrete physical operators: BroadcastHashJoin and ShuffledHashJoin. a8420ca [Michael Armbrust] Copy records in executeCollect to avoid issues with mutable rows. --- .../sql/catalyst/expressions/Projection.scala | 8 +- .../catalyst/plans/logical/BaseRelation.scala | 1 - .../scala/org/apache/spark/sql/SQLConf.scala | 17 ++ .../org/apache/spark/sql/SQLContext.scala | 6 +- .../spark/sql/execution/SparkPlan.scala | 15 +- .../spark/sql/execution/SparkStrategies.scala | 54 ++++- .../spark/sql/execution/basicOperators.scala | 1 - .../apache/spark/sql/execution/joins.scala | 219 +++++++++++------- .../spark/sql/parquet/ParquetRelation.scala | 5 +- .../org/apache/spark/sql/DslQuerySuite.scala | 99 -------- .../org/apache/spark/sql/JoinSuite.scala | 173 ++++++++++++++ .../org/apache/spark/sql/QueryTest.scala | 4 +- .../spark/sql/execution/PlannerSuite.scala | 17 -- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 7 +- 15 files changed, 395 insertions(+), 233 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index a9e976c3abaf2..2c71d2c7b3563 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -45,8 +45,10 @@ class Projection(expressions: Seq[Expression]) extends (Row => Row) { * that schema. * * In contrast to a normal projection, a MutableProjection reuses the same underlying row object - * each time an input row is added. This significatly reduces the cost of calcuating the - * projection, but means that it is not safe + * each time an input row is added. This significantly reduces the cost of calculating the + * projection, but means that it is not safe to hold on to a reference to a [[Row]] after `next()` + * has been called on the [[Iterator]] that produced it. Instead, the user must call `Row.copy()` + * and hold on to the returned [[Row]] before calling `next()`. */ case class MutableProjection(expressions: Seq[Expression]) extends (Row => Row) { def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) = @@ -67,7 +69,7 @@ case class MutableProjection(expressions: Seq[Expression]) extends (Row => Row) } /** - * A mutable wrapper that makes two rows appear appear as a single concatenated row. Designed to + * A mutable wrapper that makes two rows appear as a single concatenated row. Designed to * be instantiated once per thread and reused. */ class JoinedRow extends Row { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala index 7c616788a3830..582334aa42590 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala @@ -21,5 +21,4 @@ abstract class BaseRelation extends LeafNode { self: Product => def tableName: String - def isPartitioned: Boolean = false } 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 b378252ba2f55..2fe7f94663996 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 @@ -29,9 +29,26 @@ import scala.collection.JavaConverters._ */ trait 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("spark.sql.shuffle.partitions", "200").toInt + /** + * Upper bound on the sizes (in bytes) of the tables qualified for the auto conversion to + * a broadcast value during the physical executions of join operations. Setting this to 0 + * effectively disables auto conversion. + * Hive setting: hive.auto.convert.join.noconditionaltask.size. + */ + 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("spark.sql.join.broadcastTables", "") + + /** ********************** SQLConf functionality methods ************ */ + @transient private val settings = java.util.Collections.synchronizedMap( new java.util.HashMap[String, String]()) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 7195f9709d5e9..7edb548678c33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -170,7 +170,11 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def registerRDDAsTable(rdd: SchemaRDD, tableName: String): Unit = { - catalog.registerTable(None, tableName, rdd.logicalPlan) + val name = tableName + val newPlan = rdd.logicalPlan transform { + case s @ SparkLogicalPlan(ExistingRdd(_, _), _) => s.copy(tableName = name) + } + catalog.registerTable(None, tableName, newPlan) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 07967fe75e882..27dc091b85812 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -23,9 +23,9 @@ import org.apache.spark.sql.{Logging, Row} import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.catalyst.plans.{QueryPlan, logical} +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.logical.BaseRelation import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.columnar.InMemoryColumnarTableScan /** * :: DeveloperApi :: @@ -66,19 +66,20 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { * linking. */ @DeveloperApi -case class SparkLogicalPlan(alreadyPlanned: SparkPlan) - extends logical.LogicalPlan with MultiInstanceRelation { +case class SparkLogicalPlan(alreadyPlanned: SparkPlan, tableName: String = "SparkLogicalPlan") + extends BaseRelation with MultiInstanceRelation { def output = alreadyPlanned.output - def references = Set.empty - def children = Nil + override def references = Set.empty + override def children = Nil override final def newInstance: this.type = { SparkLogicalPlan( alreadyPlanned match { case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) case _ => sys.error("Multiple instance of the same relation detected.") - }).asInstanceOf[this.type] + }, tableName) + .asInstanceOf[this.type] } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index bd8ae4cddef89..3cd29967d1cd5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -21,10 +21,10 @@ import org.apache.spark.sql.{SQLContext, execution} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan} import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.parquet._ import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} +import org.apache.spark.sql.parquet._ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SQLContext#SparkPlanner => @@ -45,14 +45,52 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } } + /** + * Uses the HashFilteredJoin pattern to find joins where at least some of the predicates can be + * evaluated by matching hash keys. + */ object HashJoin extends Strategy with PredicateHelper { + private[this] def broadcastHashJoin( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + left: LogicalPlan, + right: LogicalPlan, + condition: Option[Expression], + side: BuildSide) = { + val broadcastHashJoin = execution.BroadcastHashJoin( + leftKeys, rightKeys, side, planLater(left), planLater(right))(sqlContext) + condition.map(Filter(_, broadcastHashJoin)).getOrElse(broadcastHashJoin) :: Nil + } + + def broadcastTables: Seq[String] = sqlContext.joinBroadcastTables.split(",").toBuffer + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - // Find inner joins where at least some predicates can be evaluated by matching hash keys - // using the HashFilteredJoin pattern. + case HashFilteredJoin( + Inner, + leftKeys, + rightKeys, + condition, + left, + right @ PhysicalOperation(_, _, b: BaseRelation)) + if broadcastTables.contains(b.tableName) => + broadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildRight) + + case HashFilteredJoin( + Inner, + leftKeys, + rightKeys, + condition, + left @ PhysicalOperation(_, _, b: BaseRelation), + right) + if broadcastTables.contains(b.tableName) => + broadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildLeft) + case HashFilteredJoin(Inner, leftKeys, rightKeys, condition, left, right) => val hashJoin = - execution.HashJoin(leftKeys, rightKeys, BuildRight, planLater(left), planLater(right)) + execution.ShuffledHashJoin( + leftKeys, rightKeys, BuildRight, planLater(left), planLater(right)) condition.map(Filter(_, hashJoin)).getOrElse(hashJoin) :: Nil + case _ => Nil } } @@ -62,10 +100,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.Aggregate(groupingExpressions, aggregateExpressions, child) => // Collect all aggregate expressions. val allAggregates = - aggregateExpressions.flatMap(_ collect { case a: AggregateExpression => a}) + aggregateExpressions.flatMap(_ collect { case a: AggregateExpression => a }) // Collect all aggregate expressions that can be computed partially. val partialAggregates = - aggregateExpressions.flatMap(_ collect { case p: PartialAggregate => p}) + aggregateExpressions.flatMap(_ collect { case p: PartialAggregate => p }) // Only do partial aggregation if supported by all aggregate expressions. if (allAggregates.size == partialAggregates.size) { @@ -242,7 +280,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.ExistingRdd(Nil, singleRowRdd) :: Nil case logical.Repartition(expressions, child) => execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil - case SparkLogicalPlan(existingPlan) => existingPlan :: Nil + case SparkLogicalPlan(existingPlan, _) => existingPlan :: Nil case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index b40d4e3a3b777..a278f1ca98476 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -205,4 +205,3 @@ object ExistingRdd { case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { override def execute() = rdd } - diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index 84bdde38b7e9e..32c5f26fe8aa0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -18,12 +18,15 @@ package org.apache.spark.sql.execution import scala.collection.mutable.{ArrayBuffer, BitSet} +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent._ +import scala.concurrent.duration._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning} +import org.apache.spark.sql.catalyst.plans.physical._ @DeveloperApi sealed abstract class BuildSide @@ -34,28 +37,19 @@ case object BuildLeft extends BuildSide @DeveloperApi case object BuildRight extends BuildSide -/** - * :: DeveloperApi :: - */ -@DeveloperApi -case class HashJoin( - leftKeys: Seq[Expression], - rightKeys: Seq[Expression], - buildSide: BuildSide, - left: SparkPlan, - right: SparkPlan) extends BinaryNode { - - override def outputPartitioning: Partitioning = left.outputPartitioning +trait HashJoin { + val leftKeys: Seq[Expression] + val rightKeys: Seq[Expression] + val buildSide: BuildSide + val left: SparkPlan + val right: SparkPlan - override def requiredChildDistribution = - ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil - - val (buildPlan, streamedPlan) = buildSide match { + lazy val (buildPlan, streamedPlan) = buildSide match { case BuildLeft => (left, right) case BuildRight => (right, left) } - val (buildKeys, streamedKeys) = buildSide match { + lazy val (buildKeys, streamedKeys) = buildSide match { case BuildLeft => (leftKeys, rightKeys) case BuildRight => (rightKeys, leftKeys) } @@ -66,73 +60,74 @@ case class HashJoin( @transient lazy val streamSideKeyGenerator = () => new MutableProjection(streamedKeys, streamedPlan.output) - def execute() = { - - buildPlan.execute().zipPartitions(streamedPlan.execute()) { (buildIter, streamIter) => - // TODO: Use Spark's HashMap implementation. - val hashTable = new java.util.HashMap[Row, ArrayBuffer[Row]]() - var currentRow: Row = null - - // Create a mapping of buildKeys -> rows - while (buildIter.hasNext) { - currentRow = buildIter.next() - val rowKey = buildSideKeyGenerator(currentRow) - if(!rowKey.anyNull) { - val existingMatchList = hashTable.get(rowKey) - val matchList = if (existingMatchList == null) { - val newMatchList = new ArrayBuffer[Row]() - hashTable.put(rowKey, newMatchList) - newMatchList - } else { - existingMatchList - } - matchList += currentRow.copy() + def joinIterators(buildIter: Iterator[Row], streamIter: Iterator[Row]): Iterator[Row] = { + // TODO: Use Spark's HashMap implementation. + + val hashTable = new java.util.HashMap[Row, ArrayBuffer[Row]]() + var currentRow: Row = null + + // Create a mapping of buildKeys -> rows + while (buildIter.hasNext) { + currentRow = buildIter.next() + val rowKey = buildSideKeyGenerator(currentRow) + if(!rowKey.anyNull) { + val existingMatchList = hashTable.get(rowKey) + val matchList = if (existingMatchList == null) { + val newMatchList = new ArrayBuffer[Row]() + hashTable.put(rowKey, newMatchList) + newMatchList + } else { + existingMatchList } + matchList += currentRow.copy() } + } - new Iterator[Row] { - private[this] var currentStreamedRow: Row = _ - private[this] var currentHashMatches: ArrayBuffer[Row] = _ - private[this] var currentMatchPosition: Int = -1 + new Iterator[Row] { + private[this] var currentStreamedRow: Row = _ + private[this] var currentHashMatches: ArrayBuffer[Row] = _ + private[this] var currentMatchPosition: Int = -1 - // Mutable per row objects. - private[this] val joinRow = new JoinedRow + // Mutable per row objects. + private[this] val joinRow = new JoinedRow - private[this] val joinKeys = streamSideKeyGenerator() + private[this] val joinKeys = streamSideKeyGenerator() - override final def hasNext: Boolean = - (currentMatchPosition != -1 && currentMatchPosition < currentHashMatches.size) || + override final def hasNext: Boolean = + (currentMatchPosition != -1 && currentMatchPosition < currentHashMatches.size) || (streamIter.hasNext && fetchNext()) - override final def next() = { - val ret = joinRow(currentStreamedRow, currentHashMatches(currentMatchPosition)) - currentMatchPosition += 1 - ret + override final def next() = { + val ret = buildSide match { + case BuildRight => joinRow(currentStreamedRow, currentHashMatches(currentMatchPosition)) + case BuildLeft => joinRow(currentHashMatches(currentMatchPosition), currentStreamedRow) } + currentMatchPosition += 1 + ret + } - /** - * Searches the streamed iterator for the next row that has at least one match in hashtable. - * - * @return true if the search is successful, and false the streamed iterator runs out of - * tuples. - */ - private final def fetchNext(): Boolean = { - currentHashMatches = null - currentMatchPosition = -1 - - while (currentHashMatches == null && streamIter.hasNext) { - currentStreamedRow = streamIter.next() - if (!joinKeys(currentStreamedRow).anyNull) { - currentHashMatches = hashTable.get(joinKeys.currentValue) - } + /** + * Searches the streamed iterator for the next row that has at least one match in hashtable. + * + * @return true if the search is successful, and false if the streamed iterator runs out of + * tuples. + */ + private final def fetchNext(): Boolean = { + currentHashMatches = null + currentMatchPosition = -1 + + while (currentHashMatches == null && streamIter.hasNext) { + currentStreamedRow = streamIter.next() + if (!joinKeys(currentStreamedRow).anyNull) { + currentHashMatches = hashTable.get(joinKeys.currentValue) } + } - if (currentHashMatches == null) { - false - } else { - currentMatchPosition = 0 - true - } + if (currentHashMatches == null) { + false + } else { + currentMatchPosition = 0 + true } } } @@ -141,32 +136,49 @@ case class HashJoin( /** * :: DeveloperApi :: - * Build the right table's join keys into a HashSet, and iteratively go through the left - * table, to find the if join keys are in the Hash set. + * Performs an inner hash join of two child relations by first shuffling the data using the join + * keys. */ @DeveloperApi -case class LeftSemiJoinHash( +case class ShuffledHashJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], + buildSide: BuildSide, left: SparkPlan, - right: SparkPlan) extends BinaryNode { + right: SparkPlan) extends BinaryNode with HashJoin { override def outputPartitioning: Partitioning = left.outputPartitioning override def requiredChildDistribution = ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil - val (buildPlan, streamedPlan) = (right, left) - val (buildKeys, streamedKeys) = (rightKeys, leftKeys) + def execute() = { + buildPlan.execute().zipPartitions(streamedPlan.execute()) { + (buildIter, streamIter) => joinIterators(buildIter, streamIter) + } + } +} - def output = left.output +/** + * :: DeveloperApi :: + * Build the right table's join keys into a HashSet, and iteratively go through the left + * table, to find the if join keys are in the Hash set. + */ +@DeveloperApi +case class LeftSemiJoinHash( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + left: SparkPlan, + right: SparkPlan) extends BinaryNode with HashJoin { - @transient lazy val buildSideKeyGenerator = new Projection(buildKeys, buildPlan.output) - @transient lazy val streamSideKeyGenerator = - () => new MutableProjection(streamedKeys, streamedPlan.output) + val buildSide = BuildRight - def execute() = { + override def requiredChildDistribution = + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + + override def output = left.output + def execute() = { buildPlan.execute().zipPartitions(streamedPlan.execute()) { (buildIter, streamIter) => val hashSet = new java.util.HashSet[Row]() var currentRow: Row = null @@ -191,6 +203,43 @@ case class LeftSemiJoinHash( } } + +/** + * :: DeveloperApi :: + * Performs an inner hash join of two child relations. When the output RDD of this operator is + * being constructed, a Spark job is asynchronously started to calculate the values for the + * broadcasted relation. This data is then placed in a Spark broadcast variable. The streamed + * relation is not shuffled. + */ +@DeveloperApi +case class BroadcastHashJoin( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + buildSide: BuildSide, + left: SparkPlan, + right: SparkPlan)(@transient sqlContext: SQLContext) extends BinaryNode with HashJoin { + + override def otherCopyArgs = sqlContext :: Nil + + override def outputPartitioning: Partitioning = left.outputPartitioning + + override def requiredChildDistribution = + UnspecifiedDistribution :: UnspecifiedDistribution :: Nil + + @transient + lazy val broadcastFuture = future { + sqlContext.sparkContext.broadcast(buildPlan.executeCollect()) + } + + def execute() = { + val broadcastRelation = Await.result(broadcastFuture, 5.minute) + + streamedPlan.execute().mapPartitions { streamedIter => + joinIterators(broadcastRelation.value.iterator, streamedIter) + } + } +} + /** * :: DeveloperApi :: * Using BroadcastNestedLoopJoin to calculate left semi join result when there's no join keys @@ -220,7 +269,6 @@ case class LeftSemiJoinBNL( .map(c => BindReferences.bindReference(c, left.output ++ right.output)) .getOrElse(Literal(true))) - def execute() = { val broadcastedRelation = sqlContext.sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) @@ -284,7 +332,6 @@ case class BroadcastNestedLoopJoin( .map(c => BindReferences.bindReference(c, left.output ++ right.output)) .getOrElse(Literal(true))) - def execute() = { val broadcastedRelation = sqlContext.sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 96c131a7f8af1..9c4771d1a9846 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -44,8 +44,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} * @param path The path to the Parquet file. */ private[sql] case class ParquetRelation( - val path: String, - @transient val conf: Option[Configuration] = None) extends LeafNode with MultiInstanceRelation { + path: String, + @transient conf: Option[Configuration] = None) extends LeafNode with MultiInstanceRelation { + self: Product => /** Schema derived from ParquetFile */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index fb599e1e01e73..e4a64a7a482b8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.test._ /* Implicits */ @@ -149,102 +148,4 @@ class DslQuerySuite extends QueryTest { test("zero count") { assert(emptyTableData.count() === 0) } - - test("inner join where, one match per row") { - checkAnswer( - upperCaseData.join(lowerCaseData, Inner).where('n === 'N), - Seq( - (1, "A", 1, "a"), - (2, "B", 2, "b"), - (3, "C", 3, "c"), - (4, "D", 4, "d") - )) - } - - test("inner join ON, one match per row") { - checkAnswer( - upperCaseData.join(lowerCaseData, Inner, Some('n === 'N)), - Seq( - (1, "A", 1, "a"), - (2, "B", 2, "b"), - (3, "C", 3, "c"), - (4, "D", 4, "d") - )) - } - - test("inner join, where, multiple matches") { - val x = testData2.where('a === 1).as('x) - val y = testData2.where('a === 1).as('y) - checkAnswer( - x.join(y).where("x.a".attr === "y.a".attr), - (1,1,1,1) :: - (1,1,1,2) :: - (1,2,1,1) :: - (1,2,1,2) :: Nil - ) - } - - test("inner join, no matches") { - val x = testData2.where('a === 1).as('x) - val y = testData2.where('a === 2).as('y) - checkAnswer( - x.join(y).where("x.a".attr === "y.a".attr), - Nil) - } - - test("big inner join, 4 matches per row") { - val bigData = testData.unionAll(testData).unionAll(testData).unionAll(testData) - val bigDataX = bigData.as('x) - val bigDataY = bigData.as('y) - - checkAnswer( - bigDataX.join(bigDataY).where("x.key".attr === "y.key".attr), - testData.flatMap( - row => Seq.fill(16)((row ++ row).toSeq)).collect().toSeq) - } - - test("cartisian product join") { - checkAnswer( - testData3.join(testData3), - (1, null, 1, null) :: - (1, null, 2, 2) :: - (2, 2, 1, null) :: - (2, 2, 2, 2) :: Nil) - } - - test("left outer join") { - checkAnswer( - upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N)), - (1, "A", 1, "a") :: - (2, "B", 2, "b") :: - (3, "C", 3, "c") :: - (4, "D", 4, "d") :: - (5, "E", null, null) :: - (6, "F", null, null) :: Nil) - } - - test("right outer join") { - checkAnswer( - lowerCaseData.join(upperCaseData, RightOuter, Some('n === 'N)), - (1, "a", 1, "A") :: - (2, "b", 2, "B") :: - (3, "c", 3, "C") :: - (4, "d", 4, "D") :: - (null, null, 5, "E") :: - (null, null, 6, "F") :: Nil) - } - - test("full outer join") { - val left = upperCaseData.where('N <= 4).as('left) - val right = upperCaseData.where('N >= 3).as('right) - - checkAnswer( - left.join(right, FullOuter, Some("left.N".attr === "right.N".attr)), - (1, "A", null, null) :: - (2, "B", null, null) :: - (3, "C", 3, "C") :: - (4, "D", 4, "D") :: - (null, null, 5, "E") :: - (null, null, 6, "F") :: Nil) - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala new file mode 100644 index 0000000000000..3d7d5eedbe8ed --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -0,0 +1,173 @@ +/* + * 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 + +import org.apache.spark.sql.TestData._ +import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter, FullOuter, Inner} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.test.TestSQLContext._ + +class JoinSuite extends QueryTest { + + // Ensures tables are loaded. + TestData + + test("equi-join is hash-join") { + val x = testData2.as('x) + val y = testData2.as('y) + val join = x.join(y, Inner, Some("x.a".attr === "y.a".attr)).queryExecution.analyzed + val planned = planner.HashJoin(join) + assert(planned.size === 1) + } + + test("plans broadcast hash join, given hints") { + + def mkTest(buildSide: BuildSide, leftTable: String, rightTable: String) = { + TestSQLContext.set("spark.sql.join.broadcastTables", + s"${if (buildSide == BuildRight) rightTable else leftTable}") + val rdd = sql(s"""SELECT * FROM $leftTable JOIN $rightTable ON key = a""") + // Using `sparkPlan` because for relevant patterns in HashJoin to be + // matched, other strategies need to be applied. + val physical = rdd.queryExecution.sparkPlan + val bhj = physical.collect { case j: BroadcastHashJoin if j.buildSide == buildSide => j } + + assert(bhj.size === 1, "planner does not pick up hint to generate broadcast hash join") + checkAnswer( + rdd, + Seq( + (1, "1", 1, 1), + (1, "1", 1, 2), + (2, "2", 2, 1), + (2, "2", 2, 2), + (3, "3", 3, 1), + (3, "3", 3, 2) + )) + } + + mkTest(BuildRight, "testData", "testData2") + mkTest(BuildLeft, "testData", "testData2") + } + + test("multiple-key equi-join is hash-join") { + val x = testData2.as('x) + val y = testData2.as('y) + val join = x.join(y, Inner, + Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)).queryExecution.analyzed + val planned = planner.HashJoin(join) + assert(planned.size === 1) + } + + test("inner join where, one match per row") { + checkAnswer( + upperCaseData.join(lowerCaseData, Inner).where('n === 'N), + Seq( + (1, "A", 1, "a"), + (2, "B", 2, "b"), + (3, "C", 3, "c"), + (4, "D", 4, "d") + )) + } + + test("inner join ON, one match per row") { + checkAnswer( + upperCaseData.join(lowerCaseData, Inner, Some('n === 'N)), + Seq( + (1, "A", 1, "a"), + (2, "B", 2, "b"), + (3, "C", 3, "c"), + (4, "D", 4, "d") + )) + } + + test("inner join, where, multiple matches") { + val x = testData2.where('a === 1).as('x) + val y = testData2.where('a === 1).as('y) + checkAnswer( + x.join(y).where("x.a".attr === "y.a".attr), + (1,1,1,1) :: + (1,1,1,2) :: + (1,2,1,1) :: + (1,2,1,2) :: Nil + ) + } + + test("inner join, no matches") { + val x = testData2.where('a === 1).as('x) + val y = testData2.where('a === 2).as('y) + checkAnswer( + x.join(y).where("x.a".attr === "y.a".attr), + Nil) + } + + test("big inner join, 4 matches per row") { + val bigData = testData.unionAll(testData).unionAll(testData).unionAll(testData) + val bigDataX = bigData.as('x) + val bigDataY = bigData.as('y) + + checkAnswer( + bigDataX.join(bigDataY).where("x.key".attr === "y.key".attr), + testData.flatMap( + row => Seq.fill(16)((row ++ row).toSeq)).collect().toSeq) + } + + test("cartisian product join") { + checkAnswer( + testData3.join(testData3), + (1, null, 1, null) :: + (1, null, 2, 2) :: + (2, 2, 1, null) :: + (2, 2, 2, 2) :: Nil) + } + + test("left outer join") { + checkAnswer( + upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N)), + (1, "A", 1, "a") :: + (2, "B", 2, "b") :: + (3, "C", 3, "c") :: + (4, "D", 4, "d") :: + (5, "E", null, null) :: + (6, "F", null, null) :: Nil) + } + + test("right outer join") { + checkAnswer( + lowerCaseData.join(upperCaseData, RightOuter, Some('n === 'N)), + (1, "a", 1, "A") :: + (2, "b", 2, "B") :: + (3, "c", 3, "C") :: + (4, "d", 4, "D") :: + (null, null, 5, "E") :: + (null, null, 6, "F") :: Nil) + } + + test("full outer join") { + val left = upperCaseData.where('N <= 4).as('left) + val right = upperCaseData.where('N >= 3).as('right) + + checkAnswer( + left.join(right, FullOuter, Some("left.N".attr === "right.N".attr)), + (1, "A", null, null) :: + (2, "B", null, null) :: + (3, "C", 3, "C") :: + (4, "D", 4, "D") :: + (null, null, 5, "E") :: + (null, null, 6, "F") :: Nil) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index ef84ead2e6e8b..8e1e1971d968b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -35,7 +35,7 @@ class QueryTest extends PlanTest { case singleItem => Seq(Seq(singleItem)) } - val isSorted = rdd.logicalPlan.collect { case s: logical.Sort => s}.nonEmpty + val isSorted = rdd.logicalPlan.collect { case s: logical.Sort => s }.nonEmpty def prepareAnswer(answer: Seq[Any]) = if (!isSorted) answer.sortBy(_.toString) else answer val sparkAnswer = try rdd.collect().toSeq catch { case e: Exception => @@ -48,7 +48,7 @@ class QueryTest extends PlanTest { """.stripMargin) } - if(prepareAnswer(convertedAnswer) != prepareAnswer(sparkAnswer)) { + if (prepareAnswer(convertedAnswer) != prepareAnswer(sparkAnswer)) { fail(s""" |Results do not match for query: |${rdd.logicalPlan} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index df6b118360d01..215618e852eb2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -57,21 +57,4 @@ class PlannerSuite extends FunSuite { val planned = PartialAggregation(query) assert(planned.isEmpty) } - - test("equi-join is hash-join") { - val x = testData2.as('x) - val y = testData2.as('y) - val join = x.join(y, Inner, Some("x.a".attr === "y.a".attr)).queryExecution.analyzed - val planned = planner.HashJoin(join) - assert(planned.size === 1) - } - - test("multiple-key equi-join is hash-join") { - val x = testData2.as('x) - val y = testData2.as('y) - val join = x.join(y, Inner, - Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)).queryExecution.analyzed - val planned = planner.HashJoin(join) - assert(planned.size === 1) - } } 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 7695242a81601..7aedfcd74189b 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 @@ -258,7 +258,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { struct.zip(fields).map { case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ))=> + case (seq: Seq[_], ArrayType(typ)) => seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") case (map: Map[_,_], MapType(kType, vType)) => map.map { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index faa30c9ae5cca..90eacf4268780 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -34,9 +34,8 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.execution.SparkLogicalPlan -import org.apache.spark.sql.hive.execution.{HiveTableScan, InsertIntoHiveTable} -import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} +import org.apache.spark.sql.columnar.InMemoryRelation +import org.apache.spark.sql.hive.execution.HiveTableScan /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -259,8 +258,6 @@ private[hive] case class MetastoreRelation new Partition(hiveQlTable, p) } - override def isPartitioned = hiveQlTable.isPartitioned - val tableDesc = new TableDesc( Class.forName(hiveQlTable.getSerializationLib).asInstanceOf[Class[Deserializer]], hiveQlTable.getInputFormatClass, From 7f196b009d26d4aed403b3c694f8b603601718e3 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 25 Jun 2014 18:41:47 -0700 Subject: [PATCH 049/124] [SPARK-2283][SQL] Reset test environment before running PruningSuite JIRA issue: [SPARK-2283](https://issues.apache.org/jira/browse/SPARK-2283) If `PruningSuite` is run right after `HiveCompatibilitySuite`, the first test case fails because `srcpart` table is cached in-memory by `HiveCompatibilitySuite`, but column pruning is not implemented for `InMemoryColumnarTableScan` operator yet. Author: Cheng Lian Closes #1221 from liancheng/spark-2283 and squashes the following commits: dc0b663 [Cheng Lian] SPARK-2283: reset test environment before running PruningSuite --- .../org/apache/spark/sql/hive/execution/PruningSuite.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 34434449a0d77..4d7c84f443879 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -26,6 +26,11 @@ import scala.collection.JavaConversions._ * A set of test cases that validate partition and column pruning. */ class PruningSuite extends HiveComparisonTest { + // Column/partition pruning is not implemented for `InMemoryColumnarTableScan` yet, need to reset + // the environment to ensure all referenced tables in this suites are not cached in-memory. + // Refer to https://issues.apache.org/jira/browse/SPARK-2283 for details. + TestHive.reset() + // Column pruning tests createPruningTest("Column pruning - with partitioned table", From b88a59a66845b8935b22f06fc96d16841ed20c94 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 25 Jun 2014 20:57:48 -0700 Subject: [PATCH 050/124] [SPARK-1749] Job cancellation when SchedulerBackend does not implement killTask This is a fixed up version of #686 (cc @markhamstra @pwendell). The last commit (the only one I authored) reflects the changes I made from Mark's original patch. Author: Mark Hamstra Author: Kay Ousterhout Closes #1219 from kayousterhout/mark-SPARK-1749 and squashes the following commits: 42dfa7e [Kay Ousterhout] Got rid of terrible double-negative name 80b3205 [Kay Ousterhout] Don't notify listeners of job failure if it wasn't successfully cancelled. d156d33 [Mark Hamstra] Do nothing in no-kill submitTasks 9312baa [Mark Hamstra] code review update cc353c8 [Mark Hamstra] scalastyle e61f7f8 [Mark Hamstra] Catch UnsupportedOperationException when DAGScheduler tries to cancel a job on a SchedulerBackend that does not implement killTask --- .../apache/spark/scheduler/DAGScheduler.scala | 30 ++++++++---- .../spark/scheduler/DAGSchedulerSuite.scala | 48 +++++++++++++++++++ 2 files changed, 69 insertions(+), 9 deletions(-) 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 b3ebaa547de0d..c8559a7a82868 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1038,7 +1038,7 @@ class DAGScheduler( private def failJobAndIndependentStages(job: ActiveJob, failureReason: String, resultStage: Option[Stage]) { val error = new SparkException(failureReason) - job.listener.jobFailed(error) + var ableToCancelStages = true val shouldInterruptThread = if (job.properties == null) false @@ -1062,18 +1062,26 @@ class DAGScheduler( // This is the only job that uses this stage, so fail the stage if it is running. val stage = stageIdToStage(stageId) if (runningStages.contains(stage)) { - taskScheduler.cancelTasks(stageId, shouldInterruptThread) - val stageInfo = stageToInfos(stage) - stageInfo.stageFailed(failureReason) - listenerBus.post(SparkListenerStageCompleted(stageToInfos(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))) + } catch { + case e: UnsupportedOperationException => + logInfo(s"Could not cancel tasks for stage $stageId", e) + ableToCancelStages = false + } } } } } - cleanupStateForJobAndIndependentStages(job, resultStage) - - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) + if (ableToCancelStages) { + job.listener.jobFailed(error) + cleanupStateForJobAndIndependentStages(job, resultStage) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) + } } /** @@ -1155,7 +1163,11 @@ private[scheduler] class DAGSchedulerActorSupervisor(dagScheduler: DAGScheduler) case x: Exception => logError("eventProcesserActor failed due to the error %s; shutting down SparkContext" .format(x.getMessage)) - dagScheduler.doCancelAllJobs() + try { + dagScheduler.doCancelAllJobs() + } catch { + case t: Throwable => logError("DAGScheduler failed to cancel all jobs.", t) + } dagScheduler.sc.stop() Stop } 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 45368328297d3..8dd2a9b9f7373 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -115,6 +115,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F sc = new SparkContext("local", "DAGSchedulerSuite") sparkListener.successfulStages.clear() sparkListener.failedStages.clear() + failure = null sc.addSparkListener(sparkListener) taskSets.clear() cancelledStages.clear() @@ -314,6 +315,53 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assertDataStructuresEmpty } + test("job cancellation no-kill backend") { + // make sure that the DAGScheduler doesn't crash when the TaskScheduler + // doesn't implement killTask() + val noKillTaskScheduler = new TaskScheduler() { + override def rootPool: Pool = null + override def schedulingMode: SchedulingMode = SchedulingMode.NONE + override def start() = {} + override def stop() = {} + override def submitTasks(taskSet: TaskSet) = { + taskSets += taskSet + } + override def cancelTasks(stageId: Int, interruptThread: Boolean) { + throw new UnsupportedOperationException + } + override def setDAGScheduler(dagScheduler: DAGScheduler) = {} + override def defaultParallelism() = 2 + } + val noKillScheduler = new DAGScheduler( + sc, + noKillTaskScheduler, + sc.listenerBus, + mapOutputTracker, + blockManagerMaster, + sc.env) { + override def runLocally(job: ActiveJob) { + // don't bother with the thread while unit testing + runLocallyWithinThread(job) + } + } + dagEventProcessTestActor = TestActorRef[DAGSchedulerEventProcessActor]( + Props(classOf[DAGSchedulerEventProcessActor], noKillScheduler))(system) + val rdd = makeRdd(1, Nil) + val jobId = submit(rdd, Array(0)) + cancel(jobId) + // Because the job wasn't actually cancelled, we shouldn't have received a failure message. + assert(failure === null) + + // When the task set completes normally, state should be correctly updated. + complete(taskSets(0), Seq((Success, 42))) + assert(results === Map(0 -> 42)) + assertDataStructuresEmpty + + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.isEmpty) + assert(sparkListener.successfulStages.contains(0)) + } + test("run trivial shuffle") { val shuffleMapRdd = makeRdd(2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) From 4a346e242c3f241c575f35536220df01ad724e23 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 25 Jun 2014 22:35:03 -0700 Subject: [PATCH 051/124] [SPARK-2284][UI] Mark all failed tasks as failures. Previously only tasks failed with ExceptionFailure reason was marked as failure. Author: Reynold Xin Closes #1224 from rxin/SPARK-2284 and squashes the following commits: be79dbd [Reynold Xin] [SPARK-2284][UI] Mark all failed tasks as failures. --- .../spark/ui/jobs/JobProgressListener.scala | 9 ++++-- .../ui/jobs/JobProgressListenerSuite.scala | 30 ++++++++++++++++++- 2 files changed, 35 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 396cbcbc8d268..bfefe4dbc4089 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -185,12 +185,15 @@ class JobProgressListener(conf: SparkConf) extends SparkListener { val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { + case org.apache.spark.Success => + stageIdToTasksComplete(sid) = stageIdToTasksComplete.getOrElse(sid, 0) + 1 + (None, Option(taskEnd.taskMetrics)) case e: ExceptionFailure => stageIdToTasksFailed(sid) = stageIdToTasksFailed.getOrElse(sid, 0) + 1 (Some(e), e.metrics) - case _ => - stageIdToTasksComplete(sid) = stageIdToTasksComplete.getOrElse(sid, 0) + 1 - (None, Option(taskEnd.taskMetrics)) + case e: org.apache.spark.TaskEndReason => + stageIdToTasksFailed(sid) = stageIdToTasksFailed.getOrElse(sid, 0) + 1 + (None, None) } stageIdToTime.getOrElseUpdate(sid, 0L) diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index c3a14f48de38e..e0fec6a068bd1 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui.jobs import org.scalatest.FunSuite import org.scalatest.Matchers -import org.apache.spark.{LocalSparkContext, SparkConf, Success} +import org.apache.spark._ import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} import org.apache.spark.scheduler._ import org.apache.spark.util.Utils @@ -101,4 +101,32 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-2", fail()) .shuffleRead == 1000) } + + test("test task success vs failure counting for different task end reasons") { + val conf = new SparkConf() + val listener = new JobProgressListener(conf) + val metrics = new TaskMetrics() + val taskInfo = new TaskInfo(1234L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL) + taskInfo.finishTime = 1 + val task = new ShuffleMapTask(0, null, null, 0, null) + val taskType = Utils.getFormattedClassName(task) + + // Go through all the failure cases to make sure we are counting them as failures. + val taskFailedReasons = Seq( + Resubmitted, + new FetchFailed(null, 0, 0, 0), + new ExceptionFailure("Exception", "description", null, None), + TaskResultLost, + TaskKilled, + ExecutorLostFailure, + UnknownReason) + for (reason <- taskFailedReasons) { + listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, reason, taskInfo, metrics)) + assert(listener.stageIdToTasksComplete.get(task.stageId) === None) + } + + // Make sure we count success as success. + listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, metrics)) + assert(listener.stageIdToTasksComplete.get(task.stageId) === Some(1)) + } } From 441cdcca64ba0b3cbaae4d4f25ebe4c4ebd46aae Mon Sep 17 00:00:00 2001 From: "Szul, Piotr" Date: Wed, 25 Jun 2014 21:55:49 -0700 Subject: [PATCH 052/124] [SPARK-2172] PySpark cannot import mllib modules in YARN-client mode Include pyspark/mllib python sources as resources in the mllib.jar. This way they will be included in the final assembly Author: Szul, Piotr Closes #1223 from piotrszul/branch-1.0 and squashes the following commits: 69d5174 [Szul, Piotr] Removed unsed resource directory src/main/resource from mllib pom f8c52a0 [Szul, Piotr] [SPARK-2172] PySpark cannot import mllib modules in YARN-client mode Include pyspark/mllib python sources as resources in the jar (cherry picked from commit fa167194ce1b5898e4d7232346c9f86b2897a722) Signed-off-by: Reynold Xin --- mllib/pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/mllib/pom.xml b/mllib/pom.xml index 878cb83dbf783..b622f96dd7901 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -84,5 +84,13 @@ scalatest-maven-plugin + + + ../python + + pyspark/mllib/*.py + + + From e4899a253728bfa7c78709a37a4837f74b72bd61 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Wed, 25 Jun 2014 23:55:31 -0700 Subject: [PATCH 053/124] [SPARK-2254] [SQL] ScalaRefection should mark primitive types as non-nullable. Author: Takuya UESHIN Closes #1193 from ueshin/issues/SPARK-2254 and squashes the following commits: cfd6088 [Takuya UESHIN] Modify ScalaRefection.schemaFor method to return nullability of Scala Type. --- .../spark/sql/catalyst/ScalaReflection.scala | 65 ++++----- .../sql/catalyst/ScalaReflectionSuite.scala | 131 ++++++++++++++++++ 2 files changed, 165 insertions(+), 31 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 196695a0a188f..ada48eaf5dc0f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -30,53 +30,56 @@ import org.apache.spark.sql.catalyst.types._ object ScalaReflection { import scala.reflect.runtime.universe._ + case class Schema(dataType: DataType, nullable: Boolean) + /** Returns a Sequence of attributes for the given case class type. */ def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match { - case s: StructType => - s.fields.map(f => AttributeReference(f.name, f.dataType, nullable = true)()) + case Schema(s: StructType, _) => + s.fields.map(f => AttributeReference(f.name, f.dataType, f.nullable)()) } - /** Returns a catalyst DataType for the given Scala Type using reflection. */ - def schemaFor[T: TypeTag]: DataType = schemaFor(typeOf[T]) + /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ + def schemaFor[T: TypeTag]: Schema = schemaFor(typeOf[T]) - /** Returns a catalyst DataType for the given Scala Type using reflection. */ - def schemaFor(tpe: `Type`): DataType = tpe match { + /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ + def schemaFor(tpe: `Type`): Schema = tpe match { case t if t <:< typeOf[Option[_]] => val TypeRef(_, _, Seq(optType)) = t - schemaFor(optType) + Schema(schemaFor(optType).dataType, nullable = true) case t if t <:< typeOf[Product] => val params = t.member("": TermName).asMethod.paramss - StructType( - params.head.map(p => - StructField(p.name.toString, schemaFor(p.typeSignature), nullable = true))) + Schema(StructType( + params.head.map { p => + val Schema(dataType, nullable) = schemaFor(p.typeSignature) + StructField(p.name.toString, dataType, nullable) + }), nullable = true) // Need to decide if we actually need a special type here. - case t if t <:< typeOf[Array[Byte]] => BinaryType + case t if t <:< typeOf[Array[Byte]] => Schema(BinaryType, nullable = true) case t if t <:< typeOf[Array[_]] => sys.error(s"Only Array[Byte] supported now, use Seq instead of $t") case t if t <:< typeOf[Seq[_]] => val TypeRef(_, _, Seq(elementType)) = t - ArrayType(schemaFor(elementType)) + Schema(ArrayType(schemaFor(elementType).dataType), nullable = true) case t if t <:< typeOf[Map[_,_]] => val TypeRef(_, _, Seq(keyType, valueType)) = t - MapType(schemaFor(keyType), schemaFor(valueType)) - case t if t <:< typeOf[String] => StringType - case t if t <:< typeOf[Timestamp] => TimestampType - case t if t <:< typeOf[BigDecimal] => DecimalType - case t if t <:< typeOf[java.lang.Integer] => IntegerType - case t if t <:< typeOf[java.lang.Long] => LongType - case t if t <:< typeOf[java.lang.Double] => DoubleType - case t if t <:< typeOf[java.lang.Float] => FloatType - case t if t <:< typeOf[java.lang.Short] => ShortType - case t if t <:< typeOf[java.lang.Byte] => ByteType - case t if t <:< typeOf[java.lang.Boolean] => BooleanType - // TODO: The following datatypes could be marked as non-nullable. - case t if t <:< definitions.IntTpe => IntegerType - case t if t <:< definitions.LongTpe => LongType - case t if t <:< definitions.DoubleTpe => DoubleType - case t if t <:< definitions.FloatTpe => FloatType - case t if t <:< definitions.ShortTpe => ShortType - case t if t <:< definitions.ByteTpe => ByteType - case t if t <:< definitions.BooleanTpe => BooleanType + Schema(MapType(schemaFor(keyType).dataType, schemaFor(valueType).dataType), nullable = true) + case t if t <:< typeOf[String] => Schema(StringType, nullable = true) + case t if t <:< typeOf[Timestamp] => Schema(TimestampType, nullable = true) + case t if t <:< typeOf[BigDecimal] => Schema(DecimalType, nullable = true) + case t if t <:< typeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) + case t if t <:< typeOf[java.lang.Long] => Schema(LongType, nullable = true) + case t if t <:< typeOf[java.lang.Double] => Schema(DoubleType, nullable = true) + case t if t <:< typeOf[java.lang.Float] => Schema(FloatType, nullable = true) + case t if t <:< typeOf[java.lang.Short] => Schema(ShortType, nullable = true) + case t if t <:< typeOf[java.lang.Byte] => Schema(ByteType, nullable = true) + case t if t <:< typeOf[java.lang.Boolean] => Schema(BooleanType, nullable = true) + case t if t <:< definitions.IntTpe => Schema(IntegerType, nullable = false) + case t if t <:< definitions.LongTpe => Schema(LongType, nullable = false) + case t if t <:< definitions.DoubleTpe => Schema(DoubleType, nullable = false) + case t if t <:< definitions.FloatTpe => Schema(FloatType, nullable = false) + case t if t <:< definitions.ShortTpe => Schema(ShortType, nullable = false) + case t if t <:< definitions.ByteTpe => Schema(ByteType, nullable = false) + case t if t <:< definitions.BooleanTpe => Schema(BooleanType, nullable = false) } implicit class CaseClassRelation[A <: Product : TypeTag](data: Seq[A]) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala new file mode 100644 index 0000000000000..489d7e9c2437f --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -0,0 +1,131 @@ +/* + * 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.catalyst + +import java.sql.Timestamp + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types._ + +case class PrimitiveData( + intField: Int, + longField: Long, + doubleField: Double, + floatField: Float, + shortField: Short, + byteField: Byte, + booleanField: Boolean) + +case class NullableData( + intField: java.lang.Integer, + longField: java.lang.Long, + doubleField: java.lang.Double, + floatField: java.lang.Float, + shortField: java.lang.Short, + byteField: java.lang.Byte, + booleanField: java.lang.Boolean, + stringField: String, + decimalField: BigDecimal, + timestampField: Timestamp, + binaryField: Array[Byte]) + +case class OptionalData( + intField: Option[Int], + longField: Option[Long], + doubleField: Option[Double], + floatField: Option[Float], + shortField: Option[Short], + byteField: Option[Byte], + booleanField: Option[Boolean]) + +case class ComplexData( + arrayField: Seq[Int], + mapField: Map[Int, String], + structField: PrimitiveData) + +class ScalaReflectionSuite extends FunSuite { + import ScalaReflection._ + + test("primitive data") { + val schema = schemaFor[PrimitiveData] + assert(schema === Schema( + StructType(Seq( + StructField("intField", IntegerType, nullable = false), + StructField("longField", LongType, nullable = false), + StructField("doubleField", DoubleType, nullable = false), + StructField("floatField", FloatType, nullable = false), + StructField("shortField", ShortType, nullable = false), + StructField("byteField", ByteType, nullable = false), + StructField("booleanField", BooleanType, nullable = false))), + nullable = true)) + } + + test("nullable data") { + val schema = schemaFor[NullableData] + assert(schema === Schema( + StructType(Seq( + StructField("intField", IntegerType, nullable = true), + StructField("longField", LongType, nullable = true), + StructField("doubleField", DoubleType, nullable = true), + StructField("floatField", FloatType, nullable = true), + StructField("shortField", ShortType, nullable = true), + StructField("byteField", ByteType, nullable = true), + StructField("booleanField", BooleanType, nullable = true), + StructField("stringField", StringType, nullable = true), + StructField("decimalField", DecimalType, nullable = true), + StructField("timestampField", TimestampType, nullable = true), + StructField("binaryField", BinaryType, nullable = true))), + nullable = true)) + } + + test("optinal data") { + val schema = schemaFor[OptionalData] + assert(schema === Schema( + StructType(Seq( + StructField("intField", IntegerType, nullable = true), + StructField("longField", LongType, nullable = true), + StructField("doubleField", DoubleType, nullable = true), + StructField("floatField", FloatType, nullable = true), + StructField("shortField", ShortType, nullable = true), + StructField("byteField", ByteType, nullable = true), + StructField("booleanField", BooleanType, nullable = true))), + nullable = true)) + } + + test("complex data") { + val schema = schemaFor[ComplexData] + assert(schema === Schema( + StructType(Seq( + StructField("arrayField", ArrayType(IntegerType), nullable = true), + StructField("mapField", MapType(IntegerType, StringType), nullable = true), + StructField( + "structField", + StructType(Seq( + StructField("intField", IntegerType, nullable = false), + StructField("longField", LongType, nullable = false), + StructField("doubleField", DoubleType, nullable = false), + StructField("floatField", FloatType, nullable = false), + StructField("shortField", ShortType, nullable = false), + StructField("byteField", ByteType, nullable = false), + StructField("booleanField", BooleanType, nullable = false))), + nullable = true))), + nullable = true)) + } +} From 48a82a827c99526b165c78d7e88faec43568a37a Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Thu, 26 Jun 2014 08:20:27 -0500 Subject: [PATCH 054/124] Remove use of spark.worker.instances spark.worker.instances was added as part of this commit: https://github.com/apache/spark/commit/1617816090e7b20124a512a43860a21232ebf511 My understanding is that SPARK_WORKER_INSTANCES is supported for backwards compatibility, but spark.worker.instances is never used (SparkSubmit.scala sets spark.executor.instances) so should not have been added. @sryza @pwendell @tgravescs LMK if I'm understanding this correctly Author: Kay Ousterhout Closes #1214 from kayousterhout/yarn_config and squashes the following commits: 3d7c491 [Kay Ousterhout] Remove use of spark.worker.instances --- .../spark/scheduler/cluster/YarnClientSchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 412dfe38d55eb..fd2694fe7278d 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -63,7 +63,7 @@ private[spark] class YarnClientSchedulerBackend( // variables. List(("--driver-memory", "SPARK_MASTER_MEMORY", "spark.master.memory"), ("--driver-memory", "SPARK_DRIVER_MEMORY", "spark.driver.memory"), - ("--num-executors", "SPARK_WORKER_INSTANCES", "spark.worker.instances"), + ("--num-executors", "SPARK_WORKER_INSTANCES", "spark.executor.instances"), ("--num-executors", "SPARK_EXECUTOR_INSTANCES", "spark.executor.instances"), ("--executor-memory", "SPARK_WORKER_MEMORY", "spark.executor.memory"), ("--executor-memory", "SPARK_EXECUTOR_MEMORY", "spark.executor.memory"), From 32a1ad75313472b1b098f7ec99335686d3fe4fc3 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 26 Jun 2014 13:37:19 -0700 Subject: [PATCH 055/124] [SPARK-2295] [SQL] Make JavaBeans nullability stricter. Author: Takuya UESHIN Closes #1235 from ueshin/issues/SPARK-2295 and squashes the following commits: 201c508 [Takuya UESHIN] Make JavaBeans nullability stricter. --- .../spark/sql/api/java/JavaSQLContext.scala | 37 +++++++++---------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index ff6deeda2394d..790d9ef22cf16 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -137,26 +137,25 @@ class JavaSQLContext(val sqlContext: SQLContext) { val fields = beanInfo.getPropertyDescriptors.filterNot(_.getName == "class") fields.map { property => - val dataType = property.getPropertyType match { - case c: Class[_] if c == classOf[java.lang.String] => StringType - case c: Class[_] if c == java.lang.Short.TYPE => ShortType - case c: Class[_] if c == java.lang.Integer.TYPE => IntegerType - case c: Class[_] if c == java.lang.Long.TYPE => LongType - case c: Class[_] if c == java.lang.Double.TYPE => DoubleType - case c: Class[_] if c == java.lang.Byte.TYPE => ByteType - case c: Class[_] if c == java.lang.Float.TYPE => FloatType - case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType - - case c: Class[_] if c == classOf[java.lang.Short] => ShortType - case c: Class[_] if c == classOf[java.lang.Integer] => IntegerType - case c: Class[_] if c == classOf[java.lang.Long] => LongType - case c: Class[_] if c == classOf[java.lang.Double] => DoubleType - case c: Class[_] if c == classOf[java.lang.Byte] => ByteType - case c: Class[_] if c == classOf[java.lang.Float] => FloatType - case c: Class[_] if c == classOf[java.lang.Boolean] => BooleanType + val (dataType, nullable) = property.getPropertyType match { + case c: Class[_] if c == classOf[java.lang.String] => (StringType, true) + case c: Class[_] if c == java.lang.Short.TYPE => (ShortType, false) + case c: Class[_] if c == java.lang.Integer.TYPE => (IntegerType, false) + case c: Class[_] if c == java.lang.Long.TYPE => (LongType, false) + case c: Class[_] if c == java.lang.Double.TYPE => (DoubleType, false) + case c: Class[_] if c == java.lang.Byte.TYPE => (ByteType, false) + case c: Class[_] if c == java.lang.Float.TYPE => (FloatType, false) + case c: Class[_] if c == java.lang.Boolean.TYPE => (BooleanType, false) + + case c: Class[_] if c == classOf[java.lang.Short] => (ShortType, true) + case c: Class[_] if c == classOf[java.lang.Integer] => (IntegerType, true) + case c: Class[_] if c == classOf[java.lang.Long] => (LongType, true) + case c: Class[_] if c == classOf[java.lang.Double] => (DoubleType, true) + case c: Class[_] if c == classOf[java.lang.Byte] => (ByteType, true) + case c: Class[_] if c == classOf[java.lang.Float] => (FloatType, true) + case c: Class[_] if c == classOf[java.lang.Boolean] => (BooleanType, true) } - // TODO: Nullability could be stricter. - AttributeReference(property.getName, dataType, nullable = true)() + AttributeReference(property.getName, dataType, nullable)() } } } From 6587ef7c1783961e6ef250afa387271a1bd6e277 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 26 Jun 2014 14:00:45 -0700 Subject: [PATCH 056/124] [SPARK-2286][UI] Report exception/errors for failed tasks that are not ExceptionFailure Also added inline doc for each TaskEndReason. Author: Reynold Xin Closes #1225 from rxin/SPARK-2286 and squashes the following commits: 6a7959d [Reynold Xin] Fix unit test failure. cf9d5eb [Reynold Xin] Merge branch 'master' into SPARK-2286 a61fae1 [Reynold Xin] Move to line above ... 38c7391 [Reynold Xin] [SPARK-2286][UI] Report exception/errors for failed tasks that are not ExceptionFailure. --- .../org/apache/spark/TaskEndReason.scala | 68 +++++++++++++++++-- .../spark/scheduler/TaskSetManager.scala | 4 +- .../spark/ui/jobs/JobProgressListener.scala | 17 ++--- .../org/apache/spark/ui/jobs/StagePage.scala | 12 +--- .../org/apache/spark/ui/jobs/StageTable.scala | 4 +- 5 files changed, 77 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index a3074916d13e7..5e8bd8c8e533a 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -30,27 +30,69 @@ import org.apache.spark.storage.BlockManagerId @DeveloperApi sealed trait TaskEndReason +/** + * :: DeveloperApi :: + * Task succeeded. + */ @DeveloperApi case object Success extends TaskEndReason +/** + * :: DeveloperApi :: + * Various possible reasons why a task failed. + */ +@DeveloperApi +sealed trait TaskFailedReason extends TaskEndReason { + /** Error message displayed in the web UI. */ + def toErrorString: String +} + +/** + * :: DeveloperApi :: + * A [[org.apache.spark.scheduler.ShuffleMapTask]] that completed successfully earlier, but we + * lost the executor before the stage completed. This means Spark needs to reschedule the task + * to be re-executed on a different executor. + */ @DeveloperApi -case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it +case object Resubmitted extends TaskFailedReason { + override def toErrorString: String = "Resubmitted (resubmitted due to lost executor)" +} +/** + * :: DeveloperApi :: + * Task failed to fetch shuffle data from a remote node. Probably means we have lost the remote + * executors the task is trying to fetch from, and thus need to rerun the previous stage. + */ @DeveloperApi case class FetchFailed( bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int) - extends TaskEndReason + extends TaskFailedReason { + override def toErrorString: String = { + val bmAddressString = if (bmAddress == null) "null" else bmAddress.toString + s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapId=$mapId, reduceId=$reduceId)" + } +} +/** + * :: DeveloperApi :: + * Task failed due to a runtime exception. This is the most common failure case and also captures + * user program exceptions. + */ @DeveloperApi case class ExceptionFailure( className: String, description: String, stackTrace: Array[StackTraceElement], metrics: Option[TaskMetrics]) - extends TaskEndReason + extends TaskFailedReason { + override def toErrorString: String = { + val stackTraceString = if (stackTrace == null) "null" else stackTrace.mkString("\n") + s"$className ($description}\n$stackTraceString" + } +} /** * :: DeveloperApi :: @@ -58,10 +100,18 @@ case class ExceptionFailure( * it was fetched. */ @DeveloperApi -case object TaskResultLost extends TaskEndReason +case object TaskResultLost extends TaskFailedReason { + override def toErrorString: String = "TaskResultLost (result lost from block manager)" +} +/** + * :: DeveloperApi :: + * Task was killed intentionally and needs to be rescheduled. + */ @DeveloperApi -case object TaskKilled extends TaskEndReason +case object TaskKilled extends TaskFailedReason { + override def toErrorString: String = "TaskKilled (killed intentionally)" +} /** * :: DeveloperApi :: @@ -69,7 +119,9 @@ case object TaskKilled extends TaskEndReason * the task crashed the JVM. */ @DeveloperApi -case object ExecutorLostFailure extends TaskEndReason +case object ExecutorLostFailure extends TaskFailedReason { + override def toErrorString: String = "ExecutorLostFailure (executor lost)" +} /** * :: DeveloperApi :: @@ -77,4 +129,6 @@ case object ExecutorLostFailure extends TaskEndReason * deserializing the task result. */ @DeveloperApi -case object UnknownReason extends TaskEndReason +case object UnknownReason extends TaskFailedReason { + override def toErrorString: String = "UnknownReason" +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index b5bcdd7e99c58..c0898f64fb0c9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -641,7 +641,9 @@ private[spark] class TaskSetManager( addPendingTask(index, readding=true) } - // Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage + // Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage. + // The reason is the next stage wouldn't be able to fetch the data from this dead executor + // so we would need to rerun these tasks on other executors. if (tasks(0).isInstanceOf[ShuffleMapTask]) { for ((tid, info) <- taskInfos if info.executorId == execId) { val index = taskInfos(tid).index diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index bfefe4dbc4089..381a5443df8b5 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -19,7 +19,7 @@ package org.apache.spark.ui.jobs import scala.collection.mutable.{HashMap, ListBuffer} -import org.apache.spark.{ExceptionFailure, SparkConf, SparkContext, Success} +import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ @@ -51,6 +51,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener { var totalShuffleRead = 0L var totalShuffleWrite = 0L + // TODO: Should probably consolidate all following into a single hash map. val stageIdToTime = HashMap[Int, Long]() val stageIdToShuffleRead = HashMap[Int, Long]() val stageIdToShuffleWrite = HashMap[Int, Long]() @@ -183,17 +184,17 @@ class JobProgressListener(conf: SparkConf) extends SparkListener { // Remove by taskId, rather than by TaskInfo, in case the TaskInfo is from storage tasksActive.remove(info.taskId) - val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = + val (errorMessage, metrics): (Option[String], Option[TaskMetrics]) = taskEnd.reason match { case org.apache.spark.Success => stageIdToTasksComplete(sid) = stageIdToTasksComplete.getOrElse(sid, 0) + 1 (None, Option(taskEnd.taskMetrics)) - case e: ExceptionFailure => + case e: ExceptionFailure => // Handle ExceptionFailure because we might have metrics stageIdToTasksFailed(sid) = stageIdToTasksFailed.getOrElse(sid, 0) + 1 - (Some(e), e.metrics) - case e: org.apache.spark.TaskEndReason => + (Some(e.toErrorString), e.metrics) + case e: TaskFailedReason => // All other failure cases stageIdToTasksFailed(sid) = stageIdToTasksFailed.getOrElse(sid, 0) + 1 - (None, None) + (Some(e.toErrorString), None) } stageIdToTime.getOrElseUpdate(sid, 0L) @@ -221,7 +222,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener { stageIdToDiskBytesSpilled(sid) += diskBytesSpilled val taskMap = stageIdToTaskData.getOrElse(sid, HashMap[Long, TaskUIData]()) - taskMap(info.taskId) = new TaskUIData(info, metrics, failureInfo) + taskMap(info.taskId) = new TaskUIData(info, metrics, errorMessage) stageIdToTaskData(sid) = taskMap } } @@ -256,7 +257,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener { case class TaskUIData( taskInfo: TaskInfo, taskMetrics: Option[TaskMetrics] = None, - exception: Option[ExceptionFailure] = None) + errorMessage: Option[String] = None) private object JobProgressListener { val DEFAULT_POOL_NAME = "default" diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 4bce472036f7d..8b65f0671bdb9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -210,10 +210,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { def taskRow(shuffleRead: Boolean, shuffleWrite: Boolean, bytesSpilled: Boolean) (taskData: TaskUIData): Seq[Node] = { - def fmtStackTrace(trace: Seq[StackTraceElement]): Seq[Node] = - trace.map(e => {e.toString}) - - taskData match { case TaskUIData(info, metrics, exception) => + taskData match { case TaskUIData(info, metrics, errorMessage) => val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) else metrics.map(_.executorRunTime).getOrElse(1L) val formatDuration = if (info.status == "RUNNING") UIUtils.formatDuration(duration) @@ -283,12 +280,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
    - {exception.map { e => - - {e.className} ({e.description})
    - {fmtStackTrace(e.stackTrace)} -
    - }.getOrElse("")} + {errorMessage.map { e =>
    {e}
    }.getOrElse("")}
    {info.index} {info.taskId}{ + if (info.speculative) s"${info.attempt} (speculative)" else info.attempt.toString + } {info.status} {info.taskLocality} {info.host} {if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""} {shuffleReadReadable} diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 7cecbfe62a382..6245b4b8023c2 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -32,6 +32,8 @@ import org.apache.spark.storage._ import org.apache.spark._ private[spark] object JsonProtocol { + // TODO: Remove this file and put JSON serialization into each individual class. + private implicit val format = DefaultFormats /** ------------------------------------------------- * @@ -194,10 +196,12 @@ private[spark] object JsonProtocol { def taskInfoToJson(taskInfo: TaskInfo): JValue = { ("Task ID" -> taskInfo.taskId) ~ ("Index" -> taskInfo.index) ~ + ("Attempt" -> taskInfo.attempt) ~ ("Launch Time" -> taskInfo.launchTime) ~ ("Executor ID" -> taskInfo.executorId) ~ ("Host" -> taskInfo.host) ~ ("Locality" -> taskInfo.taskLocality.toString) ~ + ("Speculative" -> taskInfo.speculative) ~ ("Getting Result Time" -> taskInfo.gettingResultTime) ~ ("Finish Time" -> taskInfo.finishTime) ~ ("Failed" -> taskInfo.failed) ~ @@ -487,16 +491,19 @@ private[spark] object JsonProtocol { def taskInfoFromJson(json: JValue): TaskInfo = { val taskId = (json \ "Task ID").extract[Long] val index = (json \ "Index").extract[Int] + val attempt = (json \ "Attempt").extractOpt[Int].getOrElse(1) val launchTime = (json \ "Launch Time").extract[Long] val executorId = (json \ "Executor ID").extract[String] val host = (json \ "Host").extract[String] val taskLocality = TaskLocality.withName((json \ "Locality").extract[String]) + val speculative = (json \ "Speculative").extractOpt[Boolean].getOrElse(false) val gettingResultTime = (json \ "Getting Result Time").extract[Long] val finishTime = (json \ "Finish Time").extract[Long] val failed = (json \ "Failed").extract[Boolean] val serializedSize = (json \ "Serialized Size").extract[Int] - val taskInfo = new TaskInfo(taskId, index, launchTime, executorId, host, taskLocality) + val taskInfo = + new TaskInfo(taskId, index, attempt, launchTime, executorId, host, taskLocality, speculative) taskInfo.gettingResultTime = gettingResultTime taskInfo.finishTime = finishTime taskInfo.failed = failed diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index e0fec6a068bd1..fa43b66c6cb5a 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -66,7 +66,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc // finish this task, should get updated shuffleRead shuffleReadMetrics.remoteBytesRead = 1000 taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) - var taskInfo = new TaskInfo(1234L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL) + var taskInfo = new TaskInfo(1234L, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 var task = new ShuffleMapTask(0, null, null, 0, null) val taskType = Utils.getFormattedClassName(task) @@ -75,7 +75,8 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc .shuffleRead == 1000) // finish a task with unknown executor-id, nothing should happen - taskInfo = new TaskInfo(1234L, 0, 1000L, "exe-unknown", "host1", TaskLocality.NODE_LOCAL) + taskInfo = + new TaskInfo(1234L, 0, 1, 1000L, "exe-unknown", "host1", TaskLocality.NODE_LOCAL, true) taskInfo.finishTime = 1 task = new ShuffleMapTask(0, null, null, 0, null) listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) @@ -84,7 +85,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc // finish this task, should get updated duration shuffleReadMetrics.remoteBytesRead = 1000 taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) - taskInfo = new TaskInfo(1235L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL) + taskInfo = new TaskInfo(1235L, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 task = new ShuffleMapTask(0, null, null, 0, null) listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) @@ -94,7 +95,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc // finish this task, should get updated duration shuffleReadMetrics.remoteBytesRead = 1000 taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) - taskInfo = new TaskInfo(1236L, 0, 0L, "exe-2", "host1", TaskLocality.NODE_LOCAL) + taskInfo = new TaskInfo(1236L, 0, 2, 0L, "exe-2", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 task = new ShuffleMapTask(0, null, null, 0, null) listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) @@ -106,7 +107,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc val conf = new SparkConf() val listener = new JobProgressListener(conf) val metrics = new TaskMetrics() - val taskInfo = new TaskInfo(1234L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL) + val taskInfo = new TaskInfo(1234L, 0, 3, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 val task = new ShuffleMapTask(0, null, null, 0, null) val taskType = Utils.getFormattedClassName(task) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 495e1b7a0a214..6c49870455873 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -35,10 +35,11 @@ class JsonProtocolSuite extends FunSuite { val stageSubmitted = SparkListenerStageSubmitted(makeStageInfo(100, 200, 300, 400L, 500L), properties) val stageCompleted = SparkListenerStageCompleted(makeStageInfo(101, 201, 301, 401L, 501L)) - val taskStart = SparkListenerTaskStart(111, makeTaskInfo(222L, 333, 444L)) - val taskGettingResult = SparkListenerTaskGettingResult(makeTaskInfo(1000L, 2000, 3000L)) + val taskStart = SparkListenerTaskStart(111, makeTaskInfo(222L, 333, 1, 444L, false)) + val taskGettingResult = + SparkListenerTaskGettingResult(makeTaskInfo(1000L, 2000, 5, 3000L, true)) val taskEnd = SparkListenerTaskEnd(1, "ShuffleMapTask", Success, - makeTaskInfo(123L, 234, 345L), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800)) + makeTaskInfo(123L, 234, 67, 345L, false), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800)) val jobStart = SparkListenerJobStart(10, Seq[Int](1, 2, 3, 4), properties) val jobEnd = SparkListenerJobEnd(20, JobSucceeded) val environmentUpdate = SparkListenerEnvironmentUpdate(Map[String, Seq[(String, String)]]( @@ -73,7 +74,7 @@ class JsonProtocolSuite extends FunSuite { test("Dependent Classes") { testRDDInfo(makeRddInfo(2, 3, 4, 5L, 6L)) testStageInfo(makeStageInfo(10, 20, 30, 40L, 50L)) - testTaskInfo(makeTaskInfo(999L, 888, 777L)) + testTaskInfo(makeTaskInfo(999L, 888, 55, 777L, false)) testTaskMetrics(makeTaskMetrics(33333L, 44444L, 55555L, 66666L, 7, 8)) testBlockManagerId(BlockManagerId("Hong", "Kong", 500, 1000)) @@ -269,10 +270,12 @@ class JsonProtocolSuite extends FunSuite { private def assertEquals(info1: TaskInfo, info2: TaskInfo) { assert(info1.taskId === info2.taskId) assert(info1.index === info2.index) + assert(info1.attempt === info2.attempt) assert(info1.launchTime === info2.launchTime) assert(info1.executorId === info2.executorId) assert(info1.host === info2.host) assert(info1.taskLocality === info2.taskLocality) + assert(info1.speculative === info2.speculative) assert(info1.gettingResultTime === info2.gettingResultTime) assert(info1.finishTime === info2.finishTime) assert(info1.failed === info2.failed) @@ -453,8 +456,8 @@ class JsonProtocolSuite extends FunSuite { new StageInfo(a, "greetings", b, rddInfos, "details") } - private def makeTaskInfo(a: Long, b: Int, c: Long) = { - new TaskInfo(a, b, c, "executor", "your kind sir", TaskLocality.NODE_LOCAL) + private def makeTaskInfo(a: Long, b: Int, c: Int, d: Long, speculative: Boolean) = { + new TaskInfo(a, b, c, d, "executor", "your kind sir", TaskLocality.NODE_LOCAL, speculative) } private def makeTaskMetrics(a: Long, b: Long, c: Long, d: Long, e: Int, f: Int) = { @@ -510,37 +513,60 @@ class JsonProtocolSuite extends FunSuite { private val taskStartJsonString = """ - {"Event":"SparkListenerTaskStart","Stage ID":111,"Task Info":{"Task ID":222, - "Index":333,"Launch Time":444,"Executor ID":"executor","Host":"your kind sir", - "Locality":"NODE_LOCAL","Getting Result Time":0,"Finish Time":0,"Failed":false, - "Serialized Size":0}} - """ + |{"Event":"SparkListenerTaskStart","Stage ID":111,"Task Info":{"Task ID":222, + |"Index":333,"Attempt":1,"Launch Time":444,"Executor ID":"executor","Host":"your kind sir", + |"Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0, + |"Failed":false,"Serialized Size":0}} + """.stripMargin private val taskGettingResultJsonString = """ - {"Event":"SparkListenerTaskGettingResult","Task Info":{"Task ID":1000,"Index": - 2000,"Launch Time":3000,"Executor ID":"executor","Host":"your kind sir", - "Locality":"NODE_LOCAL","Getting Result Time":0,"Finish Time":0,"Failed":false, - "Serialized Size":0}} - """ + |{"Event":"SparkListenerTaskGettingResult","Task Info": + | {"Task ID":1000,"Index":2000,"Attempt":5,"Launch Time":3000,"Executor ID":"executor", + | "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":true,"Getting Result Time":0, + | "Finish Time":0,"Failed":false,"Serialized Size":0 + | } + |} + """.stripMargin private val taskEndJsonString = """ - {"Event":"SparkListenerTaskEnd","Stage ID":1,"Task Type":"ShuffleMapTask", - "Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":123,"Index": - 234,"Launch Time":345,"Executor ID":"executor","Host":"your kind sir", - "Locality":"NODE_LOCAL","Getting Result Time":0,"Finish Time":0,"Failed": - false,"Serialized Size":0},"Task Metrics":{"Host Name":"localhost", - "Executor Deserialize Time":300,"Executor Run Time":400,"Result Size":500, - "JVM GC Time":600,"Result Serialization Time":700,"Memory Bytes Spilled": - 800,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Shuffle Finish Time": - 900,"Total Blocks Fetched":1500,"Remote Blocks Fetched":800,"Local Blocks Fetched": - 700,"Fetch Wait Time":900,"Remote Bytes Read":1000},"Shuffle Write Metrics": - {"Shuffle Bytes Written":1200,"Shuffle Write Time":1500},"Updated Blocks": - [{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":true,"Use Memory":true, - "Use Tachyon":false,"Deserialized":false,"Replication":2},"Memory Size":0,"Tachyon Size":0, - "Disk Size":0}}]}} - """ + |{"Event":"SparkListenerTaskEnd","Stage ID":1,"Task Type":"ShuffleMapTask", + |"Task End Reason":{"Reason":"Success"}, + |"Task Info":{ + | "Task ID":123,"Index":234,"Attempt":67,"Launch Time":345,"Executor ID":"executor", + | "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":false, + | "Getting Result Time":0,"Finish Time":0,"Failed":false,"Serialized Size":0 + |}, + |"Task Metrics":{ + | "Host Name":"localhost","Executor Deserialize Time":300,"Executor Run Time":400, + | "Result Size":500,"JVM GC Time":600,"Result Serialization Time":700, + | "Memory Bytes Spilled":800,"Disk Bytes Spilled":0, + | "Shuffle Read Metrics":{ + | "Shuffle Finish Time":900, + | "Total Blocks Fetched":1500, + | "Remote Blocks Fetched":800, + | "Local Blocks Fetched":700, + | "Fetch Wait Time":900, + | "Remote Bytes Read":1000 + | }, + | "Shuffle Write Metrics":{ + | "Shuffle Bytes Written":1200, + | "Shuffle Write Time":1500}, + | "Updated Blocks":[ + | {"Block ID":"rdd_0_0", + | "Status":{ + | "Storage Level":{ + | "Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":false, + | "Replication":2 + | }, + | "Memory Size":0,"Tachyon Size":0,"Disk Size":0 + | } + | } + | ] + | } + |} + """.stripMargin private val jobStartJsonString = """ From c23f5db32b3bd4d965d56e5df684a3b814a91cd6 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 26 Jun 2014 21:46:55 -0700 Subject: [PATCH 062/124] [SPARK-2251] fix concurrency issues in random sampler The following code is very likely to throw an exception: ~~~ val rdd = sc.parallelize(0 until 111, 10).sample(false, 0.1) rdd.zip(rdd).count() ~~~ because the same random number generator is used in compute partitions. Author: Xiangrui Meng Closes #1229 from mengxr/fix-sample and squashes the following commits: f1ee3d7 [Xiangrui Meng] fix concurrency issues in random sampler --- .../spark/util/random/RandomSampler.scala | 24 +++++++++---------- .../rdd/PartitionwiseSampledRDDSuite.scala | 18 ++++++++++---- .../util/random/RandomSamplerSuite.scala | 18 +++++++++----- 3 files changed, 38 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index 247f10173f1e9..32c5fdad75e58 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -54,17 +54,17 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable */ @DeveloperApi class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) - (implicit random: Random = new XORShiftRandom) extends RandomSampler[T, T] { - def this(ratio: Double)(implicit random: Random = new XORShiftRandom) - = this(0.0d, ratio)(random) + private[random] var rng: Random = new XORShiftRandom - override def setSeed(seed: Long) = random.setSeed(seed) + def this(ratio: Double) = this(0.0d, ratio) + + override def setSeed(seed: Long) = rng.setSeed(seed) override def sample(items: Iterator[T]): Iterator[T] = { items.filter { item => - val x = random.nextDouble() + val x = rng.nextDouble() (x >= lb && x < ub) ^ complement } } @@ -72,7 +72,7 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) /** * Return a sampler that is the complement of the range specified of the current sampler. */ - def cloneComplement(): BernoulliSampler[T] = new BernoulliSampler[T](lb, ub, !complement) + def cloneComplement(): BernoulliSampler[T] = new BernoulliSampler[T](lb, ub, !complement) override def clone = new BernoulliSampler[T](lb, ub, complement) } @@ -81,21 +81,21 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) * :: DeveloperApi :: * A sampler based on values drawn from Poisson distribution. * - * @param poisson a Poisson random number generator + * @param mean Poisson mean * @tparam T item type */ @DeveloperApi -class PoissonSampler[T](mean: Double) - (implicit var poisson: Poisson = new Poisson(mean, new DRand)) - extends RandomSampler[T, T] { +class PoissonSampler[T](mean: Double) extends RandomSampler[T, T] { + + private[random] var rng = new Poisson(mean, new DRand) override def setSeed(seed: Long) { - poisson = new Poisson(mean, new DRand(seed.toInt)) + rng = new Poisson(mean, new DRand(seed.toInt)) } override def sample(items: Iterator[T]): Iterator[T] = { items.flatMap { item => - val count = poisson.nextInt() + val count = rng.nextInt() if (count == 0) { Iterator.empty } else { diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala index 00c273df63b29..5dd8de319a654 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.rdd import org.scalatest.FunSuite import org.apache.spark.SharedSparkContext -import org.apache.spark.util.random.RandomSampler +import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, RandomSampler} /** a sampler that outputs its seed */ class MockSampler extends RandomSampler[Long, Long] { @@ -32,7 +32,7 @@ class MockSampler extends RandomSampler[Long, Long] { } override def sample(items: Iterator[Long]): Iterator[Long] = { - return Iterator(s) + Iterator(s) } override def clone = new MockSampler @@ -40,11 +40,21 @@ class MockSampler extends RandomSampler[Long, Long] { class PartitionwiseSampledRDDSuite extends FunSuite with SharedSparkContext { - test("seedDistribution") { + test("seed distribution") { val rdd = sc.makeRDD(Array(1L, 2L, 3L, 4L), 2) val sampler = new MockSampler val sample = new PartitionwiseSampledRDD[Long, Long](rdd, sampler, 0L) - assert(sample.distinct.count == 2, "Seeds must be different.") + assert(sample.distinct().count == 2, "Seeds must be different.") + } + + test("concurrency") { + // SPARK-2251: zip with self computes each partition twice. + // We want to make sure there are no concurrency issues. + val rdd = sc.parallelize(0 until 111, 10) + for (sampler <- Seq(new BernoulliSampler[Int](0.5), new PoissonSampler[Int](0.5))) { + val sampled = new PartitionwiseSampledRDD[Int, Int](rdd, sampler) + sampled.zip(sampled).count() + } } } diff --git a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala index e166787f17544..36877476e708e 100644 --- a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala @@ -42,7 +42,8 @@ class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } } whenExecuting(random) { - val sampler = new BernoulliSampler[Int](0.25, 0.55)(random) + val sampler = new BernoulliSampler[Int](0.25, 0.55) + sampler.rng = random assert(sampler.sample(a.iterator).toList == List(3, 4, 5)) } } @@ -54,7 +55,8 @@ class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } } whenExecuting(random) { - val sampler = new BernoulliSampler[Int](0.25, 0.55, true)(random) + val sampler = new BernoulliSampler[Int](0.25, 0.55, true) + sampler.rng = random assert(sampler.sample(a.iterator).toList === List(1, 2, 6, 7, 8, 9)) } } @@ -66,7 +68,8 @@ class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } } whenExecuting(random) { - val sampler = new BernoulliSampler[Int](0.35)(random) + val sampler = new BernoulliSampler[Int](0.35) + sampler.rng = random assert(sampler.sample(a.iterator).toList == List(1, 2, 3)) } } @@ -78,7 +81,8 @@ class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } } whenExecuting(random) { - val sampler = new BernoulliSampler[Int](0.25, 0.55, true)(random) + val sampler = new BernoulliSampler[Int](0.25, 0.55, true) + sampler.rng = random assert(sampler.sample(a.iterator).toList == List(1, 2, 6, 7, 8, 9)) } } @@ -88,7 +92,8 @@ class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar random.setSeed(10L) } whenExecuting(random) { - val sampler = new BernoulliSampler[Int](0.2)(random) + val sampler = new BernoulliSampler[Int](0.2) + sampler.rng = random sampler.setSeed(10L) } } @@ -100,7 +105,8 @@ class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } } whenExecuting(poisson) { - val sampler = new PoissonSampler[Int](0.2)(poisson) + val sampler = new PoissonSampler[Int](0.2) + sampler.rng = poisson assert(sampler.sample(a.iterator).toList == List(2, 3, 3, 5, 6)) } } From 18f29b96c7e0948f5f504e522e5aa8a8d1ab163e Mon Sep 17 00:00:00 2001 From: witgo Date: Thu, 26 Jun 2014 21:59:21 -0700 Subject: [PATCH 063/124] SPARK-2181:The keys for sorting the columns of Executor page in SparkUI are incorrect Author: witgo Closes #1135 from witgo/SPARK-2181 and squashes the following commits: 39dad90 [witgo] The keys for sorting the columns of Executor page in SparkUI are incorrect --- .../org/apache/spark/ui/exec/ExecutorsPage.scala | 8 +++++--- .../org/apache/spark/ui/jobs/ExecutorTable.scala | 12 +++++++----- .../org/apache/spark/ui/storage/StoragePage.scala | 8 +++++--- 3 files changed, 17 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 6cb43c02b8f08..2d8c3b949c1ac 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -79,6 +79,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { val maximumMemory = values("Maximum Memory") val memoryUsed = values("Memory Used") val diskUsed = values("Disk Used") + // scalastyle:off
    {values("Executor ID")} {values("Address")}{values("Failed Tasks")} {values("Complete Tasks")} {values("Total Tasks")}{Utils.msDurationToString(values("Task Time").toLong)}{Utils.bytesToString(values("Shuffle Read").toLong)}{Utils.bytesToString(values("Shuffle Write").toLong)}{Utils.msDurationToString(values("Task Time").toLong)}{Utils.bytesToString(values("Shuffle Read").toLong)}{Utils.bytesToString(values("Shuffle Write").toLong)}
    {k} {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")}{UIUtils.formatDuration(v.taskTime)}{UIUtils.formatDuration(v.taskTime)} {v.failedTasks + v.succeededTasks} {v.failedTasks} {v.succeededTasks}{Utils.bytesToString(v.shuffleRead)}{Utils.bytesToString(v.shuffleWrite)}{Utils.bytesToString(v.memoryBytesSpilled)}{Utils.bytesToString(v.diskBytesSpilled)}{Utils.bytesToString(v.shuffleRead)}{Utils.bytesToString(v.shuffleWrite)}{Utils.bytesToString(v.memoryBytesSpilled)}{Utils.bytesToString(v.diskBytesSpilled)}
    @@ -59,9 +60,10 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { {rdd.numCachedPartitions} {"%.0f%%".format(rdd.numCachedPartitions * 100.0 / rdd.numPartitions)}{Utils.bytesToString(rdd.memSize)}{Utils.bytesToString(rdd.tachyonSize)}{Utils.bytesToString(rdd.diskSize)}{Utils.bytesToString(rdd.memSize)}{Utils.bytesToString(rdd.tachyonSize)}{Utils.bytesToString(rdd.diskSize)}
    {s.failureReason.getOrElse("")}
    {s.failureReason.getOrElse("")}
    {values("Complete Tasks")} {values("Total Tasks")} {Utils.msDurationToString(values("Task Time").toLong)}{Utils.bytesToString(values("Input Bytes").toLong)} {Utils.bytesToString(values("Shuffle Read").toLong)} {Utils.bytesToString(values("Shuffle Write").toLong)}
    Total Tasks Failed Tasks Succeeded TasksInput Bytes Shuffle Read Shuffle Write Shuffle Spill (Memory){v.failedTasks + v.succeededTasks} {v.failedTasks} {v.succeededTasks}{Utils.bytesToString(v.inputBytes)} {Utils.bytesToString(v.shuffleRead)} {Utils.bytesToString(v.shuffleWrite)} {Utils.bytesToString(v.memoryBytesSpilled)} + {inputReadable} + {shuffleReadReadable} {writeTimeReadable} {memoryBytesSpilledReadable} Submitted Duration Tasks: Succeeded/TotalInput Shuffle Read Shuffle Write {makeProgressBar(startedTasks, completedTasks, failedTasks, totalTasks)} {inputRead} {shuffleRead} {shuffleWrite} Return an array with the first n elements of the dataset. Note that this is currently not executed in parallel. Instead, the driver program computes all the elements.
    takeSample(withReplacement, num, seed) Return an array with a random sample of num elements of the dataset, with or without replacement, using the given random number generator seed. takeSample(withReplacement, num, [seed]) Return an array with a random sample of num elements of the dataset, with or without replacement, optionally pre-specifying a random number generator seed.
    takeOrdered(n, [ordering])
    spark.yarn.max.executor.failures2*numExecutorsnumExecutors * 2, with minimum of 3 The maximum number of executor failures before failing the application.
    a",k.leadingWhitespace=3===b.firstChild.nodeType,k.tbody=!b.getElementsByTagName("tbody").length,k.htmlSerialize=!!b.getElementsByTagName("link").length,k.html5Clone="<:nav>"!==y.createElement("nav").cloneNode(!0).outerHTML,a.type="checkbox",a.checked=!0,c.appendChild(a),k.appendChecked=a.checked,b.innerHTML="",k.noCloneChecked=!!b.cloneNode(!0).lastChild.defaultValue,c.appendChild(b),b.innerHTML="",k.checkClone=b.cloneNode(!0).cloneNode(!0).lastChild.checked,k.noCloneEvent=!0,b.attachEvent&&(b.attachEvent("onclick",function(){k.noCloneEvent=!1}),b.cloneNode(!0).click()),null==k.deleteExpando){k.deleteExpando=!0;try{delete b.test}catch(d){k.deleteExpando=!1}}}(),function(){var b,c,d=y.createElement("div");for(b in{submit:!0,change:!0,focusin:!0})c="on"+b,(k[b+"Bubbles"]=c in a)||(d.setAttribute(c,"t"),k[b+"Bubbles"]=d.attributes[c].expando===!1);d=null}();var X=/^(?:input|select|textarea)$/i,Y=/^key/,Z=/^(?:mouse|pointer|contextmenu)|click/,$=/^(?:focusinfocus|focusoutblur)$/,_=/^([^.]*)(?:\.(.+)|)$/;function ab(){return!0}function bb(){return!1}function cb(){try{return y.activeElement}catch(a){}}m.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,n,o,p,q,r=m._data(a);if(r){c.handler&&(i=c,c=i.handler,e=i.selector),c.guid||(c.guid=m.guid++),(g=r.events)||(g=r.events={}),(k=r.handle)||(k=r.handle=function(a){return typeof m===K||a&&m.event.triggered===a.type?void 0:m.event.dispatch.apply(k.elem,arguments)},k.elem=a),b=(b||"").match(E)||[""],h=b.length;while(h--)f=_.exec(b[h])||[],o=q=f[1],p=(f[2]||"").split(".").sort(),o&&(j=m.event.special[o]||{},o=(e?j.delegateType:j.bindType)||o,j=m.event.special[o]||{},l=m.extend({type:o,origType:q,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&m.expr.match.needsContext.test(e),namespace:p.join(".")},i),(n=g[o])||(n=g[o]=[],n.delegateCount=0,j.setup&&j.setup.call(a,d,p,k)!==!1||(a.addEventListener?a.addEventListener(o,k,!1):a.attachEvent&&a.attachEvent("on"+o,k))),j.add&&(j.add.call(a,l),l.handler.guid||(l.handler.guid=c.guid)),e?n.splice(n.delegateCount++,0,l):n.push(l),m.event.global[o]=!0);a=null}},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,n,o,p,q,r=m.hasData(a)&&m._data(a);if(r&&(k=r.events)){b=(b||"").match(E)||[""],j=b.length;while(j--)if(h=_.exec(b[j])||[],o=q=h[1],p=(h[2]||"").split(".").sort(),o){l=m.event.special[o]||{},o=(d?l.delegateType:l.bindType)||o,n=k[o]||[],h=h[2]&&new RegExp("(^|\\.)"+p.join("\\.(?:.*\\.|)")+"(\\.|$)"),i=f=n.length;while(f--)g=n[f],!e&&q!==g.origType||c&&c.guid!==g.guid||h&&!h.test(g.namespace)||d&&d!==g.selector&&("**"!==d||!g.selector)||(n.splice(f,1),g.selector&&n.delegateCount--,l.remove&&l.remove.call(a,g));i&&!n.length&&(l.teardown&&l.teardown.call(a,p,r.handle)!==!1||m.removeEvent(a,o,r.handle),delete k[o])}else for(o in k)m.event.remove(a,o+b[j],c,d,!0);m.isEmptyObject(k)&&(delete r.handle,m._removeData(a,"events"))}},trigger:function(b,c,d,e){var f,g,h,i,k,l,n,o=[d||y],p=j.call(b,"type")?b.type:b,q=j.call(b,"namespace")?b.namespace.split("."):[];if(h=l=d=d||y,3!==d.nodeType&&8!==d.nodeType&&!$.test(p+m.event.triggered)&&(p.indexOf(".")>=0&&(q=p.split("."),p=q.shift(),q.sort()),g=p.indexOf(":")<0&&"on"+p,b=b[m.expando]?b:new m.Event(p,"object"==typeof b&&b),b.isTrigger=e?2:3,b.namespace=q.join("."),b.namespace_re=b.namespace?new RegExp("(^|\\.)"+q.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,b.result=void 0,b.target||(b.target=d),c=null==c?[b]:m.makeArray(c,[b]),k=m.event.special[p]||{},e||!k.trigger||k.trigger.apply(d,c)!==!1)){if(!e&&!k.noBubble&&!m.isWindow(d)){for(i=k.delegateType||p,$.test(i+p)||(h=h.parentNode);h;h=h.parentNode)o.push(h),l=h;l===(d.ownerDocument||y)&&o.push(l.defaultView||l.parentWindow||a)}n=0;while((h=o[n++])&&!b.isPropagationStopped())b.type=n>1?i:k.bindType||p,f=(m._data(h,"events")||{})[b.type]&&m._data(h,"handle"),f&&f.apply(h,c),f=g&&h[g],f&&f.apply&&m.acceptData(h)&&(b.result=f.apply(h,c),b.result===!1&&b.preventDefault());if(b.type=p,!e&&!b.isDefaultPrevented()&&(!k._default||k._default.apply(o.pop(),c)===!1)&&m.acceptData(d)&&g&&d[p]&&!m.isWindow(d)){l=d[g],l&&(d[g]=null),m.event.triggered=p;try{d[p]()}catch(r){}m.event.triggered=void 0,l&&(d[g]=l)}return b.result}},dispatch:function(a){a=m.event.fix(a);var b,c,e,f,g,h=[],i=d.call(arguments),j=(m._data(this,"events")||{})[a.type]||[],k=m.event.special[a.type]||{};if(i[0]=a,a.delegateTarget=this,!k.preDispatch||k.preDispatch.call(this,a)!==!1){h=m.event.handlers.call(this,a,j),b=0;while((f=h[b++])&&!a.isPropagationStopped()){a.currentTarget=f.elem,g=0;while((e=f.handlers[g++])&&!a.isImmediatePropagationStopped())(!a.namespace_re||a.namespace_re.test(e.namespace))&&(a.handleObj=e,a.data=e.data,c=((m.event.special[e.origType]||{}).handle||e.handler).apply(f.elem,i),void 0!==c&&(a.result=c)===!1&&(a.preventDefault(),a.stopPropagation()))}return k.postDispatch&&k.postDispatch.call(this,a),a.result}},handlers:function(a,b){var c,d,e,f,g=[],h=b.delegateCount,i=a.target;if(h&&i.nodeType&&(!a.button||"click"!==a.type))for(;i!=this;i=i.parentNode||this)if(1===i.nodeType&&(i.disabled!==!0||"click"!==a.type)){for(e=[],f=0;h>f;f++)d=b[f],c=d.selector+" ",void 0===e[c]&&(e[c]=d.needsContext?m(c,this).index(i)>=0:m.find(c,this,null,[i]).length),e[c]&&e.push(d);e.length&&g.push({elem:i,handlers:e})}return h]","i"),hb=/^\s+/,ib=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,jb=/<([\w:]+)/,kb=/\s*$/g,rb={option:[1,""],legend:[1,"
    ","
    "],area:[1,"",""],param:[1,"",""],thead:[1,"","
    "],tr:[2,"","
    "],col:[2,"","
    "],td:[3,"","
    "],_default:k.htmlSerialize?[0,"",""]:[1,"X
    ","
    "]},sb=db(y),tb=sb.appendChild(y.createElement("div"));rb.optgroup=rb.option,rb.tbody=rb.tfoot=rb.colgroup=rb.caption=rb.thead,rb.th=rb.td;function ub(a,b){var c,d,e=0,f=typeof a.getElementsByTagName!==K?a.getElementsByTagName(b||"*"):typeof a.querySelectorAll!==K?a.querySelectorAll(b||"*"):void 0;if(!f)for(f=[],c=a.childNodes||a;null!=(d=c[e]);e++)!b||m.nodeName(d,b)?f.push(d):m.merge(f,ub(d,b));return void 0===b||b&&m.nodeName(a,b)?m.merge([a],f):f}function vb(a){W.test(a.type)&&(a.defaultChecked=a.checked)}function wb(a,b){return m.nodeName(a,"table")&&m.nodeName(11!==b.nodeType?b:b.firstChild,"tr")?a.getElementsByTagName("tbody")[0]||a.appendChild(a.ownerDocument.createElement("tbody")):a}function xb(a){return a.type=(null!==m.find.attr(a,"type"))+"/"+a.type,a}function yb(a){var b=pb.exec(a.type);return b?a.type=b[1]:a.removeAttribute("type"),a}function zb(a,b){for(var c,d=0;null!=(c=a[d]);d++)m._data(c,"globalEval",!b||m._data(b[d],"globalEval"))}function Ab(a,b){if(1===b.nodeType&&m.hasData(a)){var c,d,e,f=m._data(a),g=m._data(b,f),h=f.events;if(h){delete g.handle,g.events={};for(c in h)for(d=0,e=h[c].length;e>d;d++)m.event.add(b,c,h[c][d])}g.data&&(g.data=m.extend({},g.data))}}function Bb(a,b){var c,d,e;if(1===b.nodeType){if(c=b.nodeName.toLowerCase(),!k.noCloneEvent&&b[m.expando]){e=m._data(b);for(d in e.events)m.removeEvent(b,d,e.handle);b.removeAttribute(m.expando)}"script"===c&&b.text!==a.text?(xb(b).text=a.text,yb(b)):"object"===c?(b.parentNode&&(b.outerHTML=a.outerHTML),k.html5Clone&&a.innerHTML&&!m.trim(b.innerHTML)&&(b.innerHTML=a.innerHTML)):"input"===c&&W.test(a.type)?(b.defaultChecked=b.checked=a.checked,b.value!==a.value&&(b.value=a.value)):"option"===c?b.defaultSelected=b.selected=a.defaultSelected:("input"===c||"textarea"===c)&&(b.defaultValue=a.defaultValue)}}m.extend({clone:function(a,b,c){var d,e,f,g,h,i=m.contains(a.ownerDocument,a);if(k.html5Clone||m.isXMLDoc(a)||!gb.test("<"+a.nodeName+">")?f=a.cloneNode(!0):(tb.innerHTML=a.outerHTML,tb.removeChild(f=tb.firstChild)),!(k.noCloneEvent&&k.noCloneChecked||1!==a.nodeType&&11!==a.nodeType||m.isXMLDoc(a)))for(d=ub(f),h=ub(a),g=0;null!=(e=h[g]);++g)d[g]&&Bb(e,d[g]);if(b)if(c)for(h=h||ub(a),d=d||ub(f),g=0;null!=(e=h[g]);g++)Ab(e,d[g]);else Ab(a,f);return d=ub(f,"script"),d.length>0&&zb(d,!i&&ub(a,"script")),d=h=e=null,f},buildFragment:function(a,b,c,d){for(var e,f,g,h,i,j,l,n=a.length,o=db(b),p=[],q=0;n>q;q++)if(f=a[q],f||0===f)if("object"===m.type(f))m.merge(p,f.nodeType?[f]:f);else if(lb.test(f)){h=h||o.appendChild(b.createElement("div")),i=(jb.exec(f)||["",""])[1].toLowerCase(),l=rb[i]||rb._default,h.innerHTML=l[1]+f.replace(ib,"<$1>")+l[2],e=l[0];while(e--)h=h.lastChild;if(!k.leadingWhitespace&&hb.test(f)&&p.push(b.createTextNode(hb.exec(f)[0])),!k.tbody){f="table"!==i||kb.test(f)?""!==l[1]||kb.test(f)?0:h:h.firstChild,e=f&&f.childNodes.length;while(e--)m.nodeName(j=f.childNodes[e],"tbody")&&!j.childNodes.length&&f.removeChild(j)}m.merge(p,h.childNodes),h.textContent="";while(h.firstChild)h.removeChild(h.firstChild);h=o.lastChild}else p.push(b.createTextNode(f));h&&o.removeChild(h),k.appendChecked||m.grep(ub(p,"input"),vb),q=0;while(f=p[q++])if((!d||-1===m.inArray(f,d))&&(g=m.contains(f.ownerDocument,f),h=ub(o.appendChild(f),"script"),g&&zb(h),c)){e=0;while(f=h[e++])ob.test(f.type||"")&&c.push(f)}return h=null,o},cleanData:function(a,b){for(var d,e,f,g,h=0,i=m.expando,j=m.cache,l=k.deleteExpando,n=m.event.special;null!=(d=a[h]);h++)if((b||m.acceptData(d))&&(f=d[i],g=f&&j[f])){if(g.events)for(e in g.events)n[e]?m.event.remove(d,e):m.removeEvent(d,e,g.handle);j[f]&&(delete j[f],l?delete d[i]:typeof d.removeAttribute!==K?d.removeAttribute(i):d[i]=null,c.push(f))}}}),m.fn.extend({text:function(a){return V(this,function(a){return void 0===a?m.text(this):this.empty().append((this[0]&&this[0].ownerDocument||y).createTextNode(a))},null,a,arguments.length)},append:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=wb(this,a);b.appendChild(a)}})},prepend:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=wb(this,a);b.insertBefore(a,b.firstChild)}})},before:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this)})},after:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this.nextSibling)})},remove:function(a,b){for(var c,d=a?m.filter(a,this):this,e=0;null!=(c=d[e]);e++)b||1!==c.nodeType||m.cleanData(ub(c)),c.parentNode&&(b&&m.contains(c.ownerDocument,c)&&zb(ub(c,"script")),c.parentNode.removeChild(c));return this},empty:function(){for(var a,b=0;null!=(a=this[b]);b++){1===a.nodeType&&m.cleanData(ub(a,!1));while(a.firstChild)a.removeChild(a.firstChild);a.options&&m.nodeName(a,"select")&&(a.options.length=0)}return this},clone:function(a,b){return a=null==a?!1:a,b=null==b?a:b,this.map(function(){return m.clone(this,a,b)})},html:function(a){return V(this,function(a){var b=this[0]||{},c=0,d=this.length;if(void 0===a)return 1===b.nodeType?b.innerHTML.replace(fb,""):void 0;if(!("string"!=typeof a||mb.test(a)||!k.htmlSerialize&&gb.test(a)||!k.leadingWhitespace&&hb.test(a)||rb[(jb.exec(a)||["",""])[1].toLowerCase()])){a=a.replace(ib,"<$1>");try{for(;d>c;c++)b=this[c]||{},1===b.nodeType&&(m.cleanData(ub(b,!1)),b.innerHTML=a);b=0}catch(e){}}b&&this.empty().append(a)},null,a,arguments.length)},replaceWith:function(){var a=arguments[0];return this.domManip(arguments,function(b){a=this.parentNode,m.cleanData(ub(this)),a&&a.replaceChild(b,this)}),a&&(a.length||a.nodeType)?this:this.remove()},detach:function(a){return this.remove(a,!0)},domManip:function(a,b){a=e.apply([],a);var c,d,f,g,h,i,j=0,l=this.length,n=this,o=l-1,p=a[0],q=m.isFunction(p);if(q||l>1&&"string"==typeof p&&!k.checkClone&&nb.test(p))return this.each(function(c){var d=n.eq(c);q&&(a[0]=p.call(this,c,d.html())),d.domManip(a,b)});if(l&&(i=m.buildFragment(a,this[0].ownerDocument,!1,this),c=i.firstChild,1===i.childNodes.length&&(i=c),c)){for(g=m.map(ub(i,"script"),xb),f=g.length;l>j;j++)d=i,j!==o&&(d=m.clone(d,!0,!0),f&&m.merge(g,ub(d,"script"))),b.call(this[j],d,j);if(f)for(h=g[g.length-1].ownerDocument,m.map(g,yb),j=0;f>j;j++)d=g[j],ob.test(d.type||"")&&!m._data(d,"globalEval")&&m.contains(h,d)&&(d.src?m._evalUrl&&m._evalUrl(d.src):m.globalEval((d.text||d.textContent||d.innerHTML||"").replace(qb,"")));i=c=null}return this}}),m.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(a,b){m.fn[a]=function(a){for(var c,d=0,e=[],g=m(a),h=g.length-1;h>=d;d++)c=d===h?this:this.clone(!0),m(g[d])[b](c),f.apply(e,c.get());return this.pushStack(e)}});var Cb,Db={};function Eb(b,c){var d,e=m(c.createElement(b)).appendTo(c.body),f=a.getDefaultComputedStyle&&(d=a.getDefaultComputedStyle(e[0]))?d.display:m.css(e[0],"display");return e.detach(),f}function Fb(a){var b=y,c=Db[a];return c||(c=Eb(a,b),"none"!==c&&c||(Cb=(Cb||m("