Skip to content

Commit

Permalink
Merge branch 'master' of git://git.apache.org/spark into fix-assembly…
Browse files Browse the repository at this point in the history
…-jarname
  • Loading branch information
sarutak committed Oct 6, 2014
2 parents 5fc1259 + c9ae79f commit b2318eb
Show file tree
Hide file tree
Showing 25 changed files with 395 additions and 219 deletions.
12 changes: 6 additions & 6 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -779,20 +779,20 @@ class SparkContext(config: SparkConf) extends Logging {
/**
* Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values
* with `+=`. Only the driver can access the accumuable's `value`.
* @tparam T accumulator type
* @tparam R type that can be added to the accumulator
* @tparam R accumulator result type
* @tparam T type that can be added to the accumulator
*/
def accumulable[T, R](initialValue: T)(implicit param: AccumulableParam[T, R]) =
def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T]) =
new Accumulable(initialValue, param)

/**
* Create an [[org.apache.spark.Accumulable]] shared variable, with a name for display in the
* Spark UI. Tasks can add values to the accumuable using the `+=` operator. Only the driver can
* access the accumuable's `value`.
* @tparam T accumulator type
* @tparam R type that can be added to the accumulator
* @tparam R accumulator result type
* @tparam T type that can be added to the accumulator
*/
def accumulable[T, R](initialValue: T, name: String)(implicit param: AccumulableParam[T, R]) =
def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T]) =
new Accumulable(initialValue, param, Some(name))

/**
Expand Down
25 changes: 15 additions & 10 deletions core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
Original file line number Diff line number Diff line change
Expand Up @@ -163,18 +163,23 @@ private[broadcast] object HttpBroadcast extends Logging {

private def write(id: Long, value: Any) {
val file = getFile(id)
val out: OutputStream = {
if (compress) {
compressionCodec.compressedOutputStream(new FileOutputStream(file))
} else {
new BufferedOutputStream(new FileOutputStream(file), bufferSize)
val fileOutputStream = new FileOutputStream(file)
try {
val out: OutputStream = {
if (compress) {
compressionCodec.compressedOutputStream(fileOutputStream)
} else {
new BufferedOutputStream(fileOutputStream, bufferSize)
}
}
val ser = SparkEnv.get.serializer.newInstance()
val serOut = ser.serializeStream(out)
serOut.writeObject(value)
serOut.close()
files += file
} finally {
fileOutputStream.close()
}
val ser = SparkEnv.get.serializer.newInstance()
val serOut = ser.serializeStream(out)
serOut.writeObject(value)
serOut.close()
files += file
}

private def read[T: ClassTag](id: Long): T = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,15 +83,21 @@ private[spark] class FileSystemPersistenceEngine(
val serialized = serializer.toBinary(value)

val out = new FileOutputStream(file)
out.write(serialized)
out.close()
try {
out.write(serialized)
} finally {
out.close()
}
}

def deserializeFromFile[T](file: File)(implicit m: Manifest[T]): T = {
val fileData = new Array[Byte](file.length().asInstanceOf[Int])
val dis = new DataInputStream(new FileInputStream(file))
dis.readFully(fileData)
dis.close()
try {
dis.readFully(fileData)
} finally {
dis.close()
}

val clazz = m.runtimeClass.asInstanceOf[Class[T]]
val serializer = serialization.serializerFor(clazz)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@ import scala.language.postfixOps

import akka.actor._
import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
import org.apache.commons.io.FileUtils

import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException}
import org.apache.spark.deploy.{ExecutorDescription, ExecutorState}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -372,6 +372,13 @@ private[spark] class MesosSchedulerBackend(
recordSlaveLost(d, slaveId, ExecutorExited(status))
}

override def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = {
driver.killTask(
TaskID.newBuilder()
.setValue(taskId.toString).build()
)
}

// TODO: query Mesos for number of cores
override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8)

Expand Down
16 changes: 15 additions & 1 deletion core/src/main/scala/org/apache/spark/storage/DiskStore.scala
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,21 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc
val startTime = System.currentTimeMillis
val file = diskManager.getFile(blockId)
val outputStream = new FileOutputStream(file)
blockManager.dataSerializeStream(blockId, outputStream, values)
try {
try {
blockManager.dataSerializeStream(blockId, outputStream, values)
} finally {
// Close outputStream here because it should be closed before file is deleted.
outputStream.close()
}
} catch {
case e: Throwable =>
if (file.exists()) {
file.delete()
}
throw e
}

val length = file.length

val timeTaken = System.currentTimeMillis - startTime
Expand Down
20 changes: 10 additions & 10 deletions core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -35,8 +35,6 @@ import scala.util.control.{ControlThrowable, NonFatal}

import com.google.common.io.Files
import com.google.common.util.concurrent.ThreadFactoryBuilder
import org.apache.commons.io.FileUtils
import org.apache.commons.io.filefilter.TrueFileFilter
import org.apache.commons.lang3.SystemUtils
import org.apache.hadoop.conf.Configuration
import org.apache.log4j.PropertyConfigurator
Expand Down Expand Up @@ -710,18 +708,20 @@ private[spark] object Utils extends Logging {
* Determines if a directory contains any files newer than cutoff seconds.
*
* @param dir must be the path to a directory, or IllegalArgumentException is thrown
* @param cutoff measured in seconds. Returns true if there are any files in dir newer than this.
* @param cutoff measured in seconds. Returns true if there are any files or directories in the
* given directory whose last modified time is later than this many seconds ago
*/
def doesDirectoryContainAnyNewFiles(dir: File, cutoff: Long): Boolean = {
val currentTimeMillis = System.currentTimeMillis
if (!dir.isDirectory) {
throw new IllegalArgumentException (dir + " is not a directory!")
} else {
val files = FileUtils.listFilesAndDirs(dir, TrueFileFilter.TRUE, TrueFileFilter.TRUE)
val cutoffTimeInMillis = (currentTimeMillis - (cutoff * 1000))
val newFiles = files.filter { _.lastModified > cutoffTimeInMillis }
newFiles.nonEmpty
throw new IllegalArgumentException("$dir is not a directory!")
}
val filesAndDirs = dir.listFiles()
val cutoffTimeInMillis = System.currentTimeMillis - (cutoff * 1000)

filesAndDirs.exists(_.lastModified() > cutoffTimeInMillis) ||
filesAndDirs.filter(_.isDirectory).exists(
subdir => doesDirectoryContainAnyNewFiles(subdir, cutoff)
)
}

/**
Expand Down
3 changes: 1 addition & 2 deletions dev/merge_spark_pr.py
Original file line number Diff line number Diff line change
Expand Up @@ -73,11 +73,10 @@ def fail(msg):


def run_cmd(cmd):
print cmd
if isinstance(cmd, list):
print " ".join(cmd)
return subprocess.check_output(cmd)
else:
print cmd
return subprocess.check_output(cmd.split(" "))


Expand Down
15 changes: 15 additions & 0 deletions docs/building-spark.md
Original file line number Diff line number Diff line change
Expand Up @@ -171,6 +171,21 @@ can be set to control the SBT build. For example:

sbt/sbt -Pyarn -Phadoop-2.3 assembly

# Testing with SBT

Some of the tests require Spark to be packaged first, so always run `sbt/sbt assembly` the first time. The following is an example of a correct (build, test) sequence:

sbt/sbt -Pyarn -Phadoop-2.3 -Phive assembly
sbt/sbt -Pyarn -Phadoop-2.3 -Phive test

To run only a specific test suite as follows:

sbt/sbt -Pyarn -Phadoop-2.3 -Phive "test-only org.apache.spark.repl.ReplSuite"

To run test suites of a specific sub project as follows:

sbt/sbt -Pyarn -Phadoop-2.3 -Phive core/test

# Speeding up Compilation with Zinc

[Zinc](https://github.com/typesafehub/zinc) is a long-running server version of SBT's incremental
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ object ScalaReflection {

/** Converts Scala objects to catalyst rows / types */
def convertToCatalyst(a: Any): Any = a match {
case o: Option[_] => o.orNull
case o: Option[_] => o.map(convertToCatalyst).orNull
case s: Seq[_] => s.map(convertToCatalyst)
case m: Map[_, _] => m.map { case (k, v) => convertToCatalyst(k) -> convertToCatalyst(v) }
case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,11 +67,12 @@ class SqlParser extends StandardTokenParsers with PackratParsers {
protected implicit def asParser(k: Keyword): Parser[String] =
lexical.allCaseVersions(k.str).map(x => x : Parser[String]).reduce(_ | _)

protected val ABS = Keyword("ABS")
protected val ALL = Keyword("ALL")
protected val AND = Keyword("AND")
protected val APPROXIMATE = Keyword("APPROXIMATE")
protected val AS = Keyword("AS")
protected val ASC = Keyword("ASC")
protected val APPROXIMATE = Keyword("APPROXIMATE")
protected val AVG = Keyword("AVG")
protected val BETWEEN = Keyword("BETWEEN")
protected val BY = Keyword("BY")
Expand All @@ -80,9 +81,9 @@ class SqlParser extends StandardTokenParsers with PackratParsers {
protected val COUNT = Keyword("COUNT")
protected val DESC = Keyword("DESC")
protected val DISTINCT = Keyword("DISTINCT")
protected val EXCEPT = Keyword("EXCEPT")
protected val FALSE = Keyword("FALSE")
protected val FIRST = Keyword("FIRST")
protected val LAST = Keyword("LAST")
protected val FROM = Keyword("FROM")
protected val FULL = Keyword("FULL")
protected val GROUP = Keyword("GROUP")
Expand All @@ -91,42 +92,42 @@ class SqlParser extends StandardTokenParsers with PackratParsers {
protected val IN = Keyword("IN")
protected val INNER = Keyword("INNER")
protected val INSERT = Keyword("INSERT")
protected val INTERSECT = Keyword("INTERSECT")
protected val INTO = Keyword("INTO")
protected val IS = Keyword("IS")
protected val JOIN = Keyword("JOIN")
protected val LAST = Keyword("LAST")
protected val LAZY = Keyword("LAZY")
protected val LEFT = Keyword("LEFT")
protected val LIKE = Keyword("LIKE")
protected val LIMIT = Keyword("LIMIT")
protected val LOWER = Keyword("LOWER")
protected val MAX = Keyword("MAX")
protected val MIN = Keyword("MIN")
protected val NOT = Keyword("NOT")
protected val NULL = Keyword("NULL")
protected val ON = Keyword("ON")
protected val OR = Keyword("OR")
protected val OVERWRITE = Keyword("OVERWRITE")
protected val LIKE = Keyword("LIKE")
protected val RLIKE = Keyword("RLIKE")
protected val UPPER = Keyword("UPPER")
protected val LOWER = Keyword("LOWER")
protected val REGEXP = Keyword("REGEXP")
protected val ORDER = Keyword("ORDER")
protected val OUTER = Keyword("OUTER")
protected val OVERWRITE = Keyword("OVERWRITE")
protected val REGEXP = Keyword("REGEXP")
protected val RIGHT = Keyword("RIGHT")
protected val RLIKE = Keyword("RLIKE")
protected val SELECT = Keyword("SELECT")
protected val SEMI = Keyword("SEMI")
protected val SQRT = Keyword("SQRT")
protected val STRING = Keyword("STRING")
protected val SUBSTR = Keyword("SUBSTR")
protected val SUBSTRING = Keyword("SUBSTRING")
protected val SUM = Keyword("SUM")
protected val TABLE = Keyword("TABLE")
protected val TIMESTAMP = Keyword("TIMESTAMP")
protected val TRUE = Keyword("TRUE")
protected val UNCACHE = Keyword("UNCACHE")
protected val UNION = Keyword("UNION")
protected val UPPER = Keyword("UPPER")
protected val WHERE = Keyword("WHERE")
protected val INTERSECT = Keyword("INTERSECT")
protected val EXCEPT = Keyword("EXCEPT")
protected val SUBSTR = Keyword("SUBSTR")
protected val SUBSTRING = Keyword("SUBSTRING")
protected val SQRT = Keyword("SQRT")
protected val ABS = Keyword("ABS")

// Use reflection to find the reserved words defined in this class.
protected val reservedWords =
Expand Down Expand Up @@ -183,17 +184,15 @@ class SqlParser extends StandardTokenParsers with PackratParsers {
}

protected lazy val cache: Parser[LogicalPlan] =
CACHE ~ TABLE ~> ident ~ opt(AS ~> select) <~ opt(";") ^^ {
case tableName ~ None =>
CacheCommand(tableName, true)
case tableName ~ Some(plan) =>
CacheTableAsSelectCommand(tableName, plan)
CACHE ~> opt(LAZY) ~ (TABLE ~> ident) ~ opt(AS ~> select) <~ opt(";") ^^ {
case isLazy ~ tableName ~ plan =>
CacheTableCommand(tableName, plan, isLazy.isDefined)
}

protected lazy val unCache: Parser[LogicalPlan] =
UNCACHE ~ TABLE ~> ident <~ opt(";") ^^ {
case tableName => CacheCommand(tableName, false)
}
case tableName => UncacheTableCommand(tableName)
}

protected lazy val projections: Parser[Seq[Expression]] = repsep(projection, ",")

Expand Down Expand Up @@ -283,7 +282,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers {
termExpression ~ ">=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => GreaterThanOrEqual(e1, e2) } |
termExpression ~ "!=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualTo(e1, e2)) } |
termExpression ~ "<>" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualTo(e1, e2)) } |
termExpression ~ BETWEEN ~ termExpression ~ AND ~ termExpression ^^ {
termExpression ~ BETWEEN ~ termExpression ~ AND ~ termExpression ^^ {
case e ~ _ ~ el ~ _ ~ eu => And(GreaterThanOrEqual(e, el), LessThanOrEqual(e, eu))
} |
termExpression ~ RLIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } |
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,7 @@ class SimpleCatalog(val caseSensitive: Boolean) extends Catalog {
tableName: String,
alias: Option[String] = None): LogicalPlan = {
val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName)
val table = tables.get(tblName).getOrElse(sys.error(s"Table Not Found: $tableName"))
val table = tables.getOrElse(tblName, 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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,9 +56,15 @@ case class ExplainCommand(plan: LogicalPlan, extended: Boolean = false) extends
}

/**
* Returned for the "CACHE TABLE tableName" and "UNCACHE TABLE tableName" command.
* Returned for the "CACHE TABLE tableName [AS SELECT ...]" command.
*/
case class CacheCommand(tableName: String, doCache: Boolean) extends Command
case class CacheTableCommand(tableName: String, plan: Option[LogicalPlan], isLazy: Boolean)
extends Command

/**
* Returned for the "UNCACHE TABLE tableName" command.
*/
case class UncacheTableCommand(tableName: String) extends Command

/**
* Returned for the "DESCRIBE [EXTENDED] [dbName.]tableName" command.
Expand All @@ -75,8 +81,3 @@ case class DescribeCommand(
AttributeReference("data_type", StringType, nullable = false)(),
AttributeReference("comment", StringType, nullable = false)())
}

/**
* Returned for the "CACHE TABLE tableName AS SELECT .." command.
*/
case class CacheTableAsSelectCommand(tableName: String, plan: LogicalPlan) extends Command
Loading

0 comments on commit b2318eb

Please sign in to comment.